View Javadoc

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