View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
3    * agreements. See the NOTICE file distributed with this work for additional information regarding
4    * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
5    * "License"); you may not use this file except in compliance with the License. You may obtain a
6    * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
7    * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
8    * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
9    * for the specific language governing permissions and limitations under the License.
10   */
11  package org.apache.hadoop.hbase.regionserver.compactions;
12  
13  import com.google.common.base.Preconditions;
14  import com.google.common.base.Predicate;
15  import com.google.common.collect.Collections2;
16  import com.google.common.collect.Lists;
17  
18  import java.io.IOException;
19  import java.util.ArrayList;
20  import java.util.Collection;
21  import java.util.List;
22  import java.util.Random;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
29  import org.apache.hadoop.hbase.regionserver.StoreFile;
30  import org.apache.hadoop.hbase.regionserver.StoreUtils;
31  
32  /**
33   * An abstract compaction policy that select files on seq id order.
34   */
35  @InterfaceAudience.Private
36  public abstract class SortedCompactionPolicy extends CompactionPolicy {
37  
38    private static final Log LOG = LogFactory.getLog(SortedCompactionPolicy.class);
39  
40    public SortedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
41      super(conf, storeConfigInfo);
42    }
43  
44    public List<StoreFile> preSelectCompactionForCoprocessor(final Collection<StoreFile> candidates,
45        final List<StoreFile> filesCompacting) {
46      return getCurrentEligibleFiles(new ArrayList<StoreFile>(candidates), filesCompacting);
47    }
48  
49    /**
50     * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on
51     *   DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction based
52     *   on seqId for data consistency.
53     * @return subset copy of candidate list that meets compaction criteria
54     */
55    public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
56        final List<StoreFile> filesCompacting, final boolean isUserCompaction,
57        final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
58      // Preliminary compaction subject to filters
59      ArrayList<StoreFile> candidateSelection = new ArrayList<StoreFile>(candidateFiles);
60      // Stuck and not compacting enough (estimate). It is not guaranteed that we will be
61      // able to compact more if stuck and compacting, because ratio policy excludes some
62      // non-compacting files from consideration during compaction (see getCurrentEligibleFiles).
63      int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
64      boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
65          >= storeConfigInfo.getBlockingFileCount();
66  
67      candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
68      LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
69          filesCompacting.size() + " compacting, " + candidateSelection.size() +
70          " eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
71  
72      // If we can't have all files, we cannot do major anyway
73      boolean isAllFiles = candidateFiles.size() == candidateSelection.size();
74      if (!(forceMajor && isAllFiles)) {
75        candidateSelection = skipLargeFiles(candidateSelection, mayUseOffPeak);
76        isAllFiles = candidateFiles.size() == candidateSelection.size();
77      }
78  
79      // Try a major compaction if this is a user-requested major compaction,
80      // or if we do not have too many files to compact and this was requested as a major compaction
81      boolean isTryingMajor = (forceMajor && isAllFiles && isUserCompaction)
82          || (((forceMajor && isAllFiles) || shouldPerformMajorCompaction(candidateSelection))
83            && (candidateSelection.size() < comConf.getMaxFilesToCompact()));
84      // Or, if there are any references among the candidates.
85      boolean isAfterSplit = StoreUtils.hasReferences(candidateSelection);
86  
87      CompactionRequest result = createCompactionRequest(candidateSelection,
88        isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
89  
90      ArrayList<StoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
91      removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
92      result.updateFiles(filesToCompact);
93  
94      isAllFiles = (candidateFiles.size() == filesToCompact.size());
95      result.setOffPeak(!filesToCompact.isEmpty() && !isAllFiles && mayUseOffPeak);
96      result.setIsMajor(isTryingMajor && isAllFiles, isAllFiles);
97  
98      return result;
99    }
100 
101   protected abstract CompactionRequest createCompactionRequest(ArrayList<StoreFile>
102     candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
103     throws IOException;
104 
105   /*
106    * @param filesToCompact Files to compact. Can be null.
107    * @return True if we should run a major compaction.
108    */
109   public abstract boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
110     throws IOException;
111 
112   /**
113    * Used calculation jitter
114    */
115   private final Random random = new Random();
116 
117   /**
118    * @param filesToCompact
119    * @return When to run next major compaction
120    */
121   public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
122     // default = 24hrs
123     long ret = comConf.getMajorCompactionPeriod();
124     if (ret > 0) {
125       // default = 20% = +/- 4.8 hrs
126       double jitterPct = comConf.getMajorCompactionJitter();
127       if (jitterPct > 0) {
128         long jitter = Math.round(ret * jitterPct);
129         // deterministic jitter avoids a major compaction storm on restart
130         Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
131         if (seed != null) {
132           // Synchronized to ensure one user of random instance at a time.
133           double rnd = -1;
134           synchronized (this) {
135             this.random.setSeed(seed);
136             rnd = this.random.nextDouble();
137           }
138           ret += jitter - Math.round(2L * jitter * rnd);
139         } else {
140           ret = 0; // If seed is null, then no storefiles == no major compaction
141         }
142       }
143     }
144     return ret;
145   }
146 
147   /**
148    * @param compactionSize Total size of some compaction
149    * @return whether this should be a large or small compaction
150    */
151   public boolean throttleCompaction(long compactionSize) {
152     return compactionSize > comConf.getThrottlePoint();
153   }
154 
155   public abstract boolean needsCompaction(final Collection<StoreFile> storeFiles,
156     final List<StoreFile> filesCompacting);
157 
158   protected ArrayList<StoreFile> getCurrentEligibleFiles(ArrayList<StoreFile> candidateFiles,
159       final List<StoreFile> filesCompacting) {
160     // candidates = all storefiles not already in compaction queue
161     if (!filesCompacting.isEmpty()) {
162       // exclude all files older than the newest file we're currently
163       // compacting. this allows us to preserve contiguity (HBASE-2856)
164       StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
165       int idx = candidateFiles.indexOf(last);
166       Preconditions.checkArgument(idx != -1);
167       candidateFiles.subList(0, idx + 1).clear();
168     }
169     return candidateFiles;
170   }
171 
172   /**
173    * @param candidates pre-filtrate
174    * @return filtered subset exclude all files above maxCompactSize
175    *   Also save all references. We MUST compact them
176    */
177   protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
178     boolean mayUseOffpeak) {
179     int pos = 0;
180     while (pos < candidates.size() && !candidates.get(pos).isReference()
181       && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize())) {
182       ++pos;
183     }
184     if (pos > 0) {
185       LOG.debug("Some files are too large. Excluding " + pos
186           + " files from compaction candidates");
187       candidates.subList(0, pos).clear();
188     }
189     return candidates;
190   }
191 
192   /**
193    * @param candidates pre-filtrate
194    * @return filtered subset exclude all bulk load files if configured
195    */
196   protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
197     candidates.removeAll(Collections2.filter(candidates, new Predicate<StoreFile>() {
198       @Override
199       public boolean apply(StoreFile input) {
200         return input.excludeFromMinorCompaction();
201       }
202     }));
203     return candidates;
204   }
205 
206   /**
207    * @param candidates pre-filtrate
208    */
209   protected void removeExcessFiles(ArrayList<StoreFile> candidates,
210       boolean isUserCompaction, boolean isMajorCompaction) {
211     int excess = candidates.size() - comConf.getMaxFilesToCompact();
212     if (excess > 0) {
213       if (isMajorCompaction && isUserCompaction) {
214         LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact()
215             + " files because of a user-requested major compaction");
216       } else {
217         LOG.debug("Too many admissible files. Excluding " + excess
218             + " files from compaction candidates");
219         candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
220       }
221     }
222   }
223 
224   /**
225    * @param candidates pre-filtrate
226    * @return filtered subset forget the compactionSelection if we don't have enough files
227    */
228   protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates,
229     int minFiles) {
230     if (candidates.size() < minFiles) {
231       if (LOG.isDebugEnabled()) {
232         LOG.debug("Not compacting files because we only have " + candidates.size()
233             + " files ready for compaction. Need " + minFiles + " to initiate.");
234       }
235       candidates.clear();
236     }
237     return candidates;
238   }
239 }