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 java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collection;
24  import java.util.List;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.regionserver.HStore;
32  import org.apache.hadoop.hbase.regionserver.RSRpcServices;
33  import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
34  import org.apache.hadoop.hbase.regionserver.StoreFile;
35  import org.apache.hadoop.hbase.regionserver.StoreUtils;
36  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
37  
38  /**
39   * The default algorithm for selecting files for compaction.
40   * Combines the compaction configuration and the provisional file selection that
41   * it's given to produce the list of suitable candidates for compaction.
42   */
43  @InterfaceAudience.Private
44  public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
45    private static final Log LOG = LogFactory.getLog(RatioBasedCompactionPolicy.class);
46  
47    public RatioBasedCompactionPolicy(Configuration conf,
48                                      StoreConfigInformation storeConfigInfo) {
49      super(conf, storeConfigInfo);
50    }
51  
52    /*
53     * @param filesToCompact Files to compact. Can be null.
54     * @return True if we should run a major compaction.
55     */
56    @Override
57    public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
58      throws IOException {
59      boolean result = false;
60      long mcTime = getNextMajorCompactTime(filesToCompact);
61      if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
62        return result;
63      }
64      // TODO: Use better method for determining stamp of last major (HBASE-2990)
65      long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
66      long now = EnvironmentEdgeManager.currentTime();
67      if (lowTimestamp > 0L && lowTimestamp < (now - mcTime)) {
68        String regionInfo;
69        if (this.storeConfigInfo != null && this.storeConfigInfo instanceof HStore) {
70          regionInfo = ((HStore)this.storeConfigInfo).getRegionInfo().getRegionNameAsString();
71        } else {
72          regionInfo = this.toString();
73        }
74        // Major compaction time has elapsed.
75        long cfTTL = HConstants.FOREVER;
76        if (this.storeConfigInfo != null) {
77           cfTTL = this.storeConfigInfo.getStoreFileTtl();
78        }
79        if (filesToCompact.size() == 1) {
80          // Single file
81          StoreFile sf = filesToCompact.iterator().next();
82          Long minTimestamp = sf.getMinimumTimestamp();
83          long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue();
84          if (sf.isMajorCompaction() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
85            float blockLocalityIndex =
86              sf.getHDFSBlockDistribution().getBlockLocalityIndex(
87              RSRpcServices.getHostname(comConf.conf, false));
88            if (blockLocalityIndex < comConf.getMinLocalityToForceCompact()) {
89              LOG.debug("Major compaction triggered on only store " + regionInfo
90                + "; to make hdfs blocks local, current blockLocalityIndex is "
91                + blockLocalityIndex + " (min " + comConf.getMinLocalityToForceCompact() + ")");
92              result = true;
93            } else {
94              LOG.debug("Skipping major compaction of " + regionInfo
95                + " because one (major) compacted file only, oldestTime " + oldest
96                + "ms is < TTL=" + cfTTL + " and blockLocalityIndex is " + blockLocalityIndex
97                + " (min " + comConf.getMinLocalityToForceCompact() + ")");
98            }
99          } else if (cfTTL != HConstants.FOREVER && oldest > cfTTL) {
100           LOG.debug("Major compaction triggered on store " + regionInfo
101             + ", because keyvalues outdated; time since last major compaction "
102             + (now - lowTimestamp) + "ms");
103           result = true;
104         }
105       } else {
106         LOG.debug("Major compaction triggered on store " + regionInfo
107           + "; time since last major compaction " + (now - lowTimestamp) + "ms");
108         result = true;
109       }
110     }
111     return result;
112   }
113 
114   @Override
115   protected CompactionRequest createCompactionRequest(ArrayList<StoreFile> candidateSelection,
116     boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
117     if (!tryingMajor) {
118       candidateSelection = filterBulk(candidateSelection);
119       candidateSelection = applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck);
120       candidateSelection = checkMinFilesCriteria(candidateSelection,
121         comConf.getMinFilesToCompact());
122     }
123     return new CompactionRequest(candidateSelection);
124   }
125 
126   /**
127     * -- Default minor compaction selection algorithm:
128     * choose CompactSelection from candidates --
129     * First exclude bulk-load files if indicated in configuration.
130     * Start at the oldest file and stop when you find the first file that
131     * meets compaction criteria:
132     * (1) a recently-flushed, small file (i.e. <= minCompactSize)
133     * OR
134     * (2) within the compactRatio of sum(newer_files)
135     * Given normal skew, any newer files will also meet this criteria
136     * <p/>
137     * Additional Note:
138     * If fileSizes.size() >> maxFilesToCompact, we will recurse on
139     * compact().  Consider the oldest files first to avoid a
140     * situation where we always compact [end-threshold,end).  Then, the
141     * last file becomes an aggregate of the previous compactions.
142     *
143     * normal skew:
144     *
145     *         older ----> newer (increasing seqID)
146     *     _
147     *    | |   _
148     *    | |  | |   _
149     *  --|-|- |-|- |-|---_-------_-------  minCompactSize
150     *    | |  | |  | |  | |  _  | |
151     *    | |  | |  | |  | | | | | |
152     *    | |  | |  | |  | | | | | |
153     * @param candidates pre-filtrate
154     * @return filtered subset
155     */
156   protected ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
157     boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
158     if (candidates.isEmpty()) {
159       return candidates;
160     }
161 
162     // we're doing a minor compaction, let's see what files are applicable
163     int start = 0;
164     double ratio = comConf.getCompactionRatio();
165     if (mayUseOffPeak) {
166       ratio = comConf.getCompactionRatioOffPeak();
167       LOG.info("Running an off-peak compaction, selection ratio = " + ratio);
168     }
169 
170     // get store file sizes for incremental compacting selection.
171     final int countOfFiles = candidates.size();
172     long[] fileSizes = new long[countOfFiles];
173     long[] sumSize = new long[countOfFiles];
174     for (int i = countOfFiles - 1; i >= 0; --i) {
175       StoreFile file = candidates.get(i);
176       fileSizes[i] = file.getReader().length();
177       // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
178       int tooFar = i + comConf.getMaxFilesToCompact() - 1;
179       sumSize[i] = fileSizes[i]
180         + ((i + 1 < countOfFiles) ? sumSize[i + 1] : 0)
181         - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
182     }
183 
184 
185     while (countOfFiles - start >= comConf.getMinFilesToCompact() &&
186       fileSizes[start] > Math.max(comConf.getMinCompactSize(),
187           (long) (sumSize[start + 1] * ratio))) {
188       ++start;
189     }
190     if (start < countOfFiles) {
191       LOG.info("Default compaction algorithm has selected " + (countOfFiles - start)
192         + " files from " + countOfFiles + " candidates");
193     } else if (mayBeStuck) {
194       // We may be stuck. Compact the latest files if we can.
195       int filesToLeave = candidates.size() - comConf.getMinFilesToCompact();
196       if (filesToLeave >= 0) {
197         start = filesToLeave;
198       }
199     }
200     candidates.subList(0, start).clear();
201     return candidates;
202   }
203 
204   /**
205    * A heuristic method to decide whether to schedule a compaction request
206    * @param storeFiles files in the store.
207    * @param filesCompacting files being scheduled to compact.
208    * @return true to schedule a request.
209    */
210   @Override
211   public boolean needsCompaction(final Collection<StoreFile> storeFiles,
212       final List<StoreFile> filesCompacting) {
213     int numCandidates = storeFiles.size() - filesCompacting.size();
214     return numCandidates >= comConf.getMinFilesToCompact();
215   }
216 
217   /**
218    * Overwrite min threshold for compaction
219    */
220   public void setMinThreshold(int minThreshold) {
221     comConf.setMinFilesToCompact(minThreshold);
222   }
223 }