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.collect.Iterators;
22  import com.google.common.collect.Lists;
23  import com.google.common.collect.PeekingIterator;
24  import com.google.common.math.LongMath;
25  
26  import java.io.IOException;
27  import java.util.ArrayList;
28  import java.util.Collection;
29  import java.util.Collections;
30  import java.util.List;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
36  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.regionserver.RSRpcServices;
39  import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
40  import org.apache.hadoop.hbase.regionserver.StoreFile;
41  import org.apache.hadoop.hbase.regionserver.StoreUtils;
42  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
43  import org.apache.hadoop.hbase.util.Pair;
44  import org.apache.hadoop.hbase.util.ReflectionUtils;
45  
46  /**
47   * HBASE-15181 This is a simple implementation of date-based tiered compaction similar to
48   * Cassandra's for the following benefits:
49   * <ol>
50   * <li>Improve date-range-based scan by structuring store files in date-based tiered layout.</li>
51   * <li>Reduce compaction overhead.</li>
52   * <li>Improve TTL efficiency.</li>
53   * </ol>
54   * Perfect fit for the use cases that:
55   * <ol>
56   * <li>has mostly date-based data write and scan and a focus on the most recent data.</li>
57   * </ol>
58   * Out-of-order writes are handled gracefully. Time range overlapping among store files is tolerated
59   * and the performance impact is minimized. Configuration can be set at hbase-site or overridden at
60   * per-table or per-column-family level by hbase shell. Design spec is at
61   * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/
62   */
63  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
64  public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
65  
66    private static final Log LOG = LogFactory.getLog(DateTieredCompactionPolicy.class);
67  
68    private final RatioBasedCompactionPolicy compactionPolicyPerWindow;
69  
70    private final CompactionWindowFactory windowFactory;
71  
72    public DateTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo)
73        throws IOException {
74      super(conf, storeConfigInfo);
75      try {
76        compactionPolicyPerWindow = ReflectionUtils.instantiateWithCustomCtor(
77          comConf.getCompactionPolicyForDateTieredWindow(),
78          new Class[] { Configuration.class, StoreConfigInformation.class },
79          new Object[] { conf, storeConfigInfo });
80      } catch (Exception e) {
81        throw new IOException("Unable to load configured compaction policy '"
82            + comConf.getCompactionPolicyForDateTieredWindow() + "'", e);
83      }
84      try {
85        windowFactory = ReflectionUtils.instantiateWithCustomCtor(
86          comConf.getDateTieredCompactionWindowFactory(),
87          new Class[] { CompactionConfiguration.class }, new Object[] { comConf });
88      } catch (Exception e) {
89        throw new IOException("Unable to load configured window factory '"
90            + comConf.getDateTieredCompactionWindowFactory() + "'", e);
91      }
92    }
93  
94    /**
95     * Heuristics for guessing whether we need minor compaction.
96     */
97    @Override
98    @InterfaceAudience.Private
99    public boolean needsCompaction(final Collection<StoreFile> storeFiles,
100       final List<StoreFile> filesCompacting) {
101     ArrayList<StoreFile> candidates = new ArrayList<StoreFile>(storeFiles);
102     try {
103       return !selectMinorCompaction(candidates, false, true).getFiles().isEmpty();
104     } catch (Exception e) {
105       LOG.error("Can not check for compaction: ", e);
106       return false;
107     }
108   }
109 
110   public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
111     throws IOException {
112     long mcTime = getNextMajorCompactTime(filesToCompact);
113     if (filesToCompact == null || mcTime == 0) {
114       if (LOG.isDebugEnabled()) {
115         LOG.debug("filesToCompact: " + filesToCompact + " mcTime: " + mcTime);
116       }
117       return false;
118     }
119 
120     // TODO: Use better method for determining stamp of last major (HBASE-2990)
121     long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact);
122     long now = EnvironmentEdgeManager.currentTime();
123     if (lowTimestamp <= 0L || lowTimestamp >= (now - mcTime)) {
124       if (LOG.isDebugEnabled()) {
125         LOG.debug("lowTimestamp: " + lowTimestamp + " lowTimestamp: " + lowTimestamp + " now: " +
126             now + " mcTime: " + mcTime); 
127       }
128       return false;
129     }
130 
131     long cfTTL = this.storeConfigInfo.getStoreFileTtl();
132     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
133     List<Long> boundaries = getCompactBoundariesForMajor(filesToCompact, now);
134     boolean[] filesInWindow = new boolean[boundaries.size()];
135 
136     for (StoreFile file: filesToCompact) {
137       Long minTimestamp = file.getMinimumTimestamp();
138       long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue();
139       if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) {
140         LOG.debug("Major compaction triggered on store " + this
141           + "; for TTL maintenance");
142         return true;
143       }
144       if (!file.isMajorCompaction() || file.isBulkLoadResult()) {
145         LOG.debug("Major compaction triggered on store " + this
146           + ", because there are new files and time since last major compaction "
147           + (now - lowTimestamp) + "ms");
148         return true;
149       }
150 
151       int lowerWindowIndex = Collections.binarySearch(boundaries,
152         minTimestamp == null ? (Long)Long.MAX_VALUE : minTimestamp);
153       int upperWindowIndex = Collections.binarySearch(boundaries,
154         file.getMaximumTimestamp() == null ? (Long)Long.MAX_VALUE : file.getMaximumTimestamp());
155       // Handle boundary conditions and negative values of binarySearch
156       lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex;
157       upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex;
158       if (lowerWindowIndex != upperWindowIndex) {
159         LOG.debug("Major compaction triggered on store " + this + "; because file "
160           + file.getPath() + " has data with timestamps cross window boundaries");
161         return true;
162       } else if (filesInWindow[upperWindowIndex]) {
163         LOG.debug("Major compaction triggered on store " + this +
164           "; because there are more than one file in some windows");
165         return true;
166       } else {
167         filesInWindow[upperWindowIndex] = true;
168       }
169       hdfsBlocksDistribution.add(file.getHDFSBlockDistribution());
170     }
171 
172     float blockLocalityIndex = hdfsBlocksDistribution
173         .getBlockLocalityIndex(RSRpcServices.getHostname(comConf.conf, false));
174     if (blockLocalityIndex < comConf.getMinLocalityToForceCompact()) {
175       LOG.debug("Major compaction triggered on store " + this
176         + "; to make hdfs blocks local, current blockLocalityIndex is "
177         + blockLocalityIndex + " (min " + comConf.getMinLocalityToForceCompact() + ")");
178       return true;
179     }
180 
181     LOG.debug("Skipping major compaction of " + this +
182       ", because the files are already major compacted");
183     return false;
184   }
185 
186   @Override
187   protected CompactionRequest createCompactionRequest(ArrayList<StoreFile> candidateSelection,
188     boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
189     CompactionRequest result = tryingMajor ? selectMajorCompaction(candidateSelection)
190       : selectMinorCompaction(candidateSelection, mayUseOffPeak, mayBeStuck);
191     if (LOG.isDebugEnabled()) {
192       LOG.debug("Generated compaction request: " + result);
193     }
194     return result;
195   }
196 
197   public CompactionRequest selectMajorCompaction(ArrayList<StoreFile> candidateSelection) {
198     long now = EnvironmentEdgeManager.currentTime();
199     return new DateTieredCompactionRequest(candidateSelection,
200       this.getCompactBoundariesForMajor(candidateSelection, now));
201   }
202 
203   /**
204    * We receive store files sorted in ascending order by seqId then scan the list of files. If the
205    * current file has a maxTimestamp older than last known maximum, treat this file as it carries
206    * the last known maximum. This way both seqId and timestamp are in the same order. If files carry
207    * the same maxTimestamps, they are ordered by seqId. We then reverse the list so they are ordered
208    * by seqId and maxTimestamp in descending order and build the time windows. All the out-of-order
209    * data into the same compaction windows, guaranteeing contiguous compaction based on sequence id.
210    */
211   public CompactionRequest selectMinorCompaction(ArrayList<StoreFile> candidateSelection,
212       boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
213     long now = EnvironmentEdgeManager.currentTime();
214     long oldestToCompact = getOldestToCompact(comConf.getDateTieredMaxStoreFileAgeMillis(), now);
215 
216     List<Pair<StoreFile, Long>> storefileMaxTimestampPairs =
217         Lists.newArrayListWithCapacity(candidateSelection.size());
218     long maxTimestampSeen = Long.MIN_VALUE;
219     for (StoreFile storeFile : candidateSelection) {
220       // if there is out-of-order data,
221       // we put them in the same window as the last file in increasing order
222       maxTimestampSeen = Math.max(maxTimestampSeen,
223         storeFile.getMaximumTimestamp() == null? Long.MIN_VALUE : storeFile.getMaximumTimestamp());
224       storefileMaxTimestampPairs.add(new Pair<StoreFile, Long>(storeFile, maxTimestampSeen));
225     }
226     Collections.reverse(storefileMaxTimestampPairs);
227 
228     CompactionWindow window = getIncomingWindow(now);
229     int minThreshold = comConf.getDateTieredIncomingWindowMin();
230     PeekingIterator<Pair<StoreFile, Long>> it =
231         Iterators.peekingIterator(storefileMaxTimestampPairs.iterator());
232     while (it.hasNext()) {
233       if (window.compareToTimestamp(oldestToCompact) < 0) {
234         break;
235       }
236       int compResult = window.compareToTimestamp(it.peek().getSecond());
237       if (compResult > 0) {
238         // If the file is too old for the window, switch to the next window
239         window = window.nextEarlierWindow();
240         minThreshold = comConf.getMinFilesToCompact();
241       } else {
242         // The file is within the target window
243         ArrayList<StoreFile> fileList = Lists.newArrayList();
244         // Add all files in the same window. For incoming window
245         // we tolerate files with future data although it is sub-optimal
246         while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) {
247           fileList.add(it.next().getFirst());
248         }
249         if (fileList.size() >= minThreshold) {
250           if (LOG.isDebugEnabled()) {
251             LOG.debug("Processing files: " + fileList + " for window: " + window);
252           }
253           DateTieredCompactionRequest request = generateCompactionRequest(fileList, window,
254             mayUseOffPeak, mayBeStuck, minThreshold);
255           if (request != null) {
256             return request;
257           }
258         }
259       }
260     }
261     // A non-null file list is expected by HStore
262     return new CompactionRequest(Collections.<StoreFile> emptyList());
263   }
264 
265   private DateTieredCompactionRequest generateCompactionRequest(ArrayList<StoreFile> storeFiles,
266       CompactionWindow window, boolean mayUseOffPeak, boolean mayBeStuck, int minThreshold)
267       throws IOException {
268     // The files has to be in ascending order for ratio-based compaction to work right
269     // and removeExcessFile to exclude youngest files.
270     Collections.reverse(storeFiles);
271 
272     // Compact everything in the window if have more files than comConf.maxBlockingFiles
273     compactionPolicyPerWindow.setMinThreshold(minThreshold);
274     ArrayList<StoreFile> storeFileSelection = mayBeStuck ? storeFiles
275       : compactionPolicyPerWindow.applyCompactionPolicy(storeFiles, mayUseOffPeak, false);
276     if (storeFileSelection != null && !storeFileSelection.isEmpty()) {
277       // If there is any file in the window excluded from compaction,
278       // only one file will be output from compaction.
279       boolean singleOutput = storeFiles.size() != storeFileSelection.size() ||
280         comConf.useDateTieredSingleOutputForMinorCompaction();
281       List<Long> boundaries = getCompactionBoundariesForMinor(window, singleOutput);
282       DateTieredCompactionRequest result = new DateTieredCompactionRequest(storeFileSelection,
283         boundaries);
284       return result;
285     }
286     return null;
287   }
288 
289   /**
290    * Return a list of boundaries for multiple compaction output
291    *   in ascending order.
292    */
293   private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) {
294     long minTimestamp = Long.MAX_VALUE;
295     for (StoreFile file : filesToCompact) {
296       minTimestamp =
297         Math.min(minTimestamp,
298           file.getMinimumTimestamp() == null ? Long.MAX_VALUE : file.getMinimumTimestamp());
299     }
300 
301     List<Long> boundaries = new ArrayList<Long>();
302 
303     // Add startMillis of all windows between now and min timestamp
304     for (CompactionWindow window = getIncomingWindow(now);
305         window.compareToTimestamp(minTimestamp) > 0;
306         window = window.nextEarlierWindow()) {
307       boundaries.add(window.startMillis());
308     }
309     boundaries.add(Long.MIN_VALUE);
310     Collections.reverse(boundaries);
311     return boundaries;
312   }
313 
314   /**
315    * @return a list of boundaries for multiple compaction output from minTimestamp to maxTimestamp.
316    */
317   private static List<Long> getCompactionBoundariesForMinor(CompactionWindow window,
318       boolean singleOutput) {
319     List<Long> boundaries = new ArrayList<Long>();
320     boundaries.add(Long.MIN_VALUE);
321     if (!singleOutput) {
322       boundaries.add(window.startMillis());
323     }
324     return boundaries;
325   }
326 
327   private CompactionWindow getIncomingWindow(long now) {
328     return windowFactory.newIncomingWindow(now);
329   }
330 
331   private static long getOldestToCompact(long maxAgeMillis, long now) {
332     try {
333       return LongMath.checkedSubtract(now, maxAgeMillis);
334     } catch (ArithmeticException ae) {
335       LOG.warn("Value for " + CompactionConfiguration.DATE_TIERED_MAX_AGE_MILLIS_KEY + ": "
336           + maxAgeMillis + ". All the files will be eligible for minor compaction.");
337       return Long.MIN_VALUE;
338     }
339   }
340 }