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      result.setAfterSplit(isAfterSplit);
90  
91      ArrayList<StoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
92      removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
93      result.updateFiles(filesToCompact);
94  
95      isAllFiles = (candidateFiles.size() == filesToCompact.size());
96      result.setOffPeak(!filesToCompact.isEmpty() && !isAllFiles && mayUseOffPeak);
97      result.setIsMajor(isTryingMajor && isAllFiles, isAllFiles);
98  
99      return result;
100   }
101 
102   protected abstract CompactionRequest createCompactionRequest(ArrayList<StoreFile>
103     candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
104     throws IOException;
105 
106   /*
107    * @param filesToCompact Files to compact. Can be null.
108    * @return True if we should run a major compaction.
109    */
110   @Override
111   public abstract boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
112     throws IOException;
113 
114   /**
115    * Used calculation jitter
116    */
117   private final Random random = new Random();
118 
119   /**
120    * @param filesToCompact
121    * @return When to run next major compaction
122    */
123   public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
124     // default = 7days
125     long ret = comConf.getMajorCompactionPeriod();
126     if (ret > 0) {
127       // default = 20% = +/- 4.8 hrs
128       double jitterPct = comConf.getMajorCompactionJitter();
129       if (jitterPct > 0) {
130         long jitter = Math.round(ret * jitterPct);
131         // deterministic jitter avoids a major compaction storm on restart
132         Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
133         if (seed != null) {
134           // Synchronized to ensure one user of random instance at a time.
135           double rnd = -1;
136           synchronized (this) {
137             this.random.setSeed(seed);
138             rnd = this.random.nextDouble();
139           }
140           ret += jitter - Math.round(2L * jitter * rnd);
141         } else {
142           ret = 0; // If seed is null, then no storefiles == no major compaction
143         }
144       }
145     }
146     return ret;
147   }
148 
149   /**
150    * @param compactionSize Total size of some compaction
151    * @return whether this should be a large or small compaction
152    */
153   @Override
154   public boolean throttleCompaction(long compactionSize) {
155     return compactionSize > comConf.getThrottlePoint();
156   }
157 
158   public abstract boolean needsCompaction(final Collection<StoreFile> storeFiles,
159     final List<StoreFile> filesCompacting);
160 
161   protected ArrayList<StoreFile> getCurrentEligibleFiles(ArrayList<StoreFile> candidateFiles,
162       final List<StoreFile> filesCompacting) {
163     // candidates = all storefiles not already in compaction queue
164     if (!filesCompacting.isEmpty()) {
165       // exclude all files older than the newest file we're currently
166       // compacting. this allows us to preserve contiguity (HBASE-2856)
167       StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
168       int idx = candidateFiles.indexOf(last);
169       Preconditions.checkArgument(idx != -1);
170       candidateFiles.subList(0, idx + 1).clear();
171     }
172     return candidateFiles;
173   }
174 
175   /**
176    * @param candidates pre-filtrate
177    * @return filtered subset exclude all files above maxCompactSize
178    *   Also save all references. We MUST compact them
179    */
180   protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
181     boolean mayUseOffpeak) {
182     int pos = 0;
183     while (pos < candidates.size() && !candidates.get(pos).isReference()
184       && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize(mayUseOffpeak))) {
185       ++pos;
186     }
187     if (pos > 0) {
188       LOG.debug("Some files are too large. Excluding " + pos
189           + " files from compaction candidates");
190       candidates.subList(0, pos).clear();
191     }
192     return candidates;
193   }
194 
195   /**
196    * @param candidates pre-filtrate
197    * @return filtered subset exclude all bulk load files if configured
198    */
199   protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
200     candidates.removeAll(Collections2.filter(candidates, new Predicate<StoreFile>() {
201       @Override
202       public boolean apply(StoreFile input) {
203         return input.excludeFromMinorCompaction();
204       }
205     }));
206     return candidates;
207   }
208 
209   /**
210    * @param candidates pre-filtrate
211    */
212   protected void removeExcessFiles(ArrayList<StoreFile> candidates,
213       boolean isUserCompaction, boolean isMajorCompaction) {
214     int excess = candidates.size() - comConf.getMaxFilesToCompact();
215     if (excess > 0) {
216       if (isMajorCompaction && isUserCompaction) {
217         LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact()
218             + " files because of a user-requested major compaction");
219       } else {
220         LOG.debug("Too many admissible files. Excluding " + excess
221             + " files from compaction candidates");
222         candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
223       }
224     }
225   }
226 
227   /**
228    * @param candidates pre-filtrate
229    * @return filtered subset forget the compactionSelection if we don't have enough files
230    */
231   protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates,
232     int minFiles) {
233     if (candidates.size() < minFiles) {
234       if (LOG.isDebugEnabled()) {
235         LOG.debug("Not compacting files because we only have " + candidates.size()
236             + " files ready for compaction. Need " + minFiles + " to initiate.");
237       }
238       candidates.clear();
239     }
240     return candidates;
241   }
242 }