View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver.compactions;
19  
20  import java.io.IOException;
21  import java.io.InterruptedIOException;
22  import java.security.PrivilegedExceptionAction;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.Collections;
26  import java.util.List;
27  import java.util.Map;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.Cell;
34  import org.apache.hadoop.hbase.CellUtil;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.KeyValueUtil;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.client.Scan;
39  import org.apache.hadoop.hbase.io.compress.Compression;
40  import org.apache.hadoop.hbase.io.hfile.HFile;
41  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
42  import org.apache.hadoop.hbase.regionserver.HStore;
43  import org.apache.hadoop.hbase.regionserver.InternalScanner;
44  import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
45  import org.apache.hadoop.hbase.regionserver.ScanType;
46  import org.apache.hadoop.hbase.regionserver.ScannerContext;
47  import org.apache.hadoop.hbase.regionserver.Store;
48  import org.apache.hadoop.hbase.regionserver.StoreFile;
49  import org.apache.hadoop.hbase.regionserver.StoreFileReader;
50  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
51  import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
52  import org.apache.hadoop.hbase.regionserver.StoreScanner;
53  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
54  import org.apache.hadoop.hbase.regionserver.compactions.Compactor.CellSink;
55  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
56  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
57  import org.apache.hadoop.hbase.security.User;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
60  import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
61  
62  import com.google.common.io.Closeables;
63  
64  /**
65   * A compactor is a compaction algorithm associated a given policy. Base class also contains
66   * reusable parts for implementing compactors (what is common and what isn't is evolving).
67   */
68  @InterfaceAudience.Private
69  public abstract class Compactor<T extends CellSink> {
70    private static final Log LOG = LogFactory.getLog(Compactor.class);
71    private static final long COMPACTION_PROGRESS_LOG_INTERVAL = 60 * 1000;
72    protected volatile CompactionProgress progress;
73    protected final Configuration conf;
74    protected final Store store;
75  
76    protected final int compactionKVMax;
77    protected final Compression.Algorithm compactionCompression;
78  
79    /** specify how many days to keep MVCC values during major compaction **/ 
80    protected final int keepSeqIdPeriod;
81  
82    //TODO: depending on Store is not good but, realistically, all compactors currently do.
83    Compactor(final Configuration conf, final Store store) {
84      this.conf = conf;
85      this.store = store;
86      this.compactionKVMax =
87        this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
88      this.compactionCompression = (this.store.getFamily() == null) ?
89          Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompressionType();
90      this.keepSeqIdPeriod = Math.max(this.conf.getInt(HConstants.KEEP_SEQID_PERIOD, 
91        HConstants.MIN_KEEP_SEQID_PERIOD), HConstants.MIN_KEEP_SEQID_PERIOD);
92    }
93  
94    public interface CellSink {
95      void append(Cell cell) throws IOException;
96    }
97  
98    protected interface CellSinkFactory<S> {
99      S createWriter(InternalScanner scanner, FileDetails fd, boolean shouldDropBehind)
100         throws IOException;
101   }
102 
103   public CompactionProgress getProgress() {
104     return this.progress;
105   }
106 
107   /** The sole reason this class exists is that java has no ref/out/pointer parameters. */
108   protected static class FileDetails {
109     /** Maximum key count after compaction (for blooms) */
110     public long maxKeyCount = 0;
111     /** Earliest put timestamp if major compaction */
112     public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
113     /** Latest put timestamp */
114     public long latestPutTs = HConstants.LATEST_TIMESTAMP;
115     /** The last key in the files we're compacting. */
116     public long maxSeqId = 0;
117     /** Latest memstore read point found in any of the involved files */
118     public long maxMVCCReadpoint = 0;
119     /** Max tags length**/
120     public int maxTagsLength = 0;
121     /** Min SeqId to keep during a major compaction **/
122     public long minSeqIdToKeep = 0;
123   }
124 
125   /**
126    * Extracts some details about the files to compact that are commonly needed by compactors.
127    * @param filesToCompact Files.
128    * @param allFiles Whether all files are included for compaction
129    * @return The result.
130    */
131   protected FileDetails getFileDetails(
132       Collection<StoreFile> filesToCompact, boolean allFiles) throws IOException {
133     FileDetails fd = new FileDetails();
134     long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - 
135       (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);  
136 
137     for (StoreFile file : filesToCompact) {
138       if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
139         // when isAllFiles is true, all files are compacted so we can calculate the smallest 
140         // MVCC value to keep
141         if(fd.minSeqIdToKeep < file.getMaxMemstoreTS()) {
142           fd.minSeqIdToKeep = file.getMaxMemstoreTS();
143         }
144       }
145       long seqNum = file.getMaxSequenceId();
146       fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
147       StoreFileReader r = file.getReader();
148       if (r == null) {
149         LOG.warn("Null reader for " + file.getPath());
150         continue;
151       }
152       // NOTE: use getEntries when compacting instead of getFilterEntries, otherwise under-sized
153       // blooms can cause progress to be miscalculated or if the user switches bloom
154       // type (e.g. from ROW to ROWCOL)
155       long keyCount = r.getEntries();
156       fd.maxKeyCount += keyCount;
157       // calculate the latest MVCC readpoint in any of the involved store files
158       Map<byte[], byte[]> fileInfo = r.loadFileInfo();
159       byte[] tmp = null;
160       // Get and set the real MVCCReadpoint for bulk loaded files, which is the
161       // SeqId number.
162       if (r.isBulkLoaded()) {
163         fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, r.getSequenceID());
164       }
165       else {
166         tmp = fileInfo.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
167         if (tmp != null) {
168           fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
169         }
170       }
171       tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
172       if (tmp != null) {
173         fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
174       }
175       // If required, calculate the earliest put timestamp of all involved storefiles.
176       // This is used to remove family delete marker during compaction.
177       long earliestPutTs = 0;
178       if (allFiles) {
179         tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
180         if (tmp == null) {
181           // There's a file with no information, must be an old one
182           // assume we have very old puts
183           fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
184         } else {
185           earliestPutTs = Bytes.toLong(tmp);
186           fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
187         }
188       }
189       tmp = fileInfo.get(StoreFile.TIMERANGE_KEY);
190       TimeRangeTracker trt = TimeRangeTracker.getTimeRangeTracker(tmp);
191       fd.latestPutTs = trt == null? HConstants.LATEST_TIMESTAMP: trt.getMax();
192       if (LOG.isDebugEnabled()) {
193         LOG.debug("Compacting " + file +
194           ", keycount=" + keyCount +
195           ", bloomtype=" + r.getBloomFilterType().toString() +
196           ", size=" + TraditionalBinaryPrefix.long2String(r.length(), "", 1) +
197           ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
198           ", seqNum=" + seqNum +
199           (allFiles ? ", earliestPutTs=" + earliestPutTs: ""));
200       }
201     }
202     return fd;
203   }
204 
205   /**
206    * Creates file scanners for compaction.
207    * @param filesToCompact Files.
208    * @return Scanners.
209    */
210   protected List<StoreFileScanner> createFileScanners(
211       final Collection<StoreFile> filesToCompact,
212       long smallestReadPoint,
213       boolean useDropBehind) throws IOException {
214     return StoreFileScanner.getScannersForStoreFiles(filesToCompact,
215         /* cache blocks = */ false,
216         /* use pread = */ false,
217         /* is compaction */ true,
218         /* use Drop Behind */ useDropBehind,
219       smallestReadPoint);
220   }
221 
222   protected long getSmallestReadPoint() {
223     return store.getSmallestReadPoint();
224   }
225 
226   protected interface InternalScannerFactory {
227 
228     ScanType getScanType(CompactionRequest request);
229 
230     InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
231         FileDetails fd, long smallestReadPoint) throws IOException;
232   }
233 
234   protected final InternalScannerFactory defaultScannerFactory = new InternalScannerFactory() {
235 
236     @Override
237     public ScanType getScanType(CompactionRequest request) {
238       return request.isRetainDeleteMarkers() ? ScanType.COMPACT_RETAIN_DELETES
239           : ScanType.COMPACT_DROP_DELETES;
240     }
241 
242     @Override
243     public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
244         FileDetails fd, long smallestReadPoint) throws IOException {
245       return Compactor.this.createScanner(store, scanners, scanType, smallestReadPoint,
246         fd.earliestPutTs);
247     }
248   };
249 
250   /**
251    * Creates a writer for a new file in a temporary directory.
252    * @param fd The file details.
253    * @return Writer for a new StoreFile in the tmp dir.
254    * @throws IOException if creation failed
255    */
256   protected StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind)
257       throws IOException {
258     // When all MVCC readpoints are 0, don't write them.
259     // See HBASE-8166, HBASE-12600, and HBASE-13389.
260     return store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
261     /* isCompaction = */true,
262     /* includeMVCCReadpoint = */fd.maxMVCCReadpoint > 0,
263     /* includesTags = */fd.maxTagsLength > 0, shouldDropBehind);
264   }
265 
266   protected List<Path> compact(final CompactionRequest request,
267       InternalScannerFactory scannerFactory, CellSinkFactory<T> sinkFactory,
268       ThroughputController throughputController, User user) throws IOException {
269     FileDetails fd = getFileDetails(request.getFiles(), request.isAllFiles());
270     this.progress = new CompactionProgress(fd.maxKeyCount);
271 
272     // Find the smallest read point across all the Scanners.
273     long smallestReadPoint = getSmallestReadPoint();
274 
275     List<StoreFileScanner> scanners;
276     Collection<StoreFile> readersToClose;
277     T writer = null;
278     if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) {
279       // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
280       // HFiles, and their readers
281       readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
282       for (StoreFile f : request.getFiles()) {
283         readersToClose.add(f.cloneForReader());
284       }
285       scanners = createFileScanners(readersToClose, smallestReadPoint,
286         store.throttleCompaction(request.getSize()));
287     } else {
288       readersToClose = Collections.emptyList();
289       scanners = createFileScanners(request.getFiles(), smallestReadPoint,
290         store.throttleCompaction(request.getSize()));
291     }
292     InternalScanner scanner = null;
293     boolean finished = false;
294     try {
295       /* Include deletes, unless we are doing a major compaction */
296       ScanType scanType = scannerFactory.getScanType(request);
297       scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
298       if (scanner == null) {
299         scanner = scannerFactory.createScanner(scanners, scanType, fd, smallestReadPoint);
300       }
301       scanner = postCreateCoprocScanner(request, scanType, scanner, user);
302       if (scanner == null) {
303         // NULL scanner returned from coprocessor hooks means skip normal processing.
304         return new ArrayList<Path>();
305       }
306       boolean cleanSeqId = false;
307       if (fd.minSeqIdToKeep > 0) {
308         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
309         cleanSeqId = true;
310       }
311       writer = sinkFactory.createWriter(scanner, fd, store.throttleCompaction(request.getSize()));
312       finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
313         throughputController, request.isAllFiles());
314       if (!finished) {
315         throw new InterruptedIOException("Aborting compaction of store " + store + " in region "
316             + store.getRegionInfo().getRegionNameAsString() + " because it was interrupted.");
317       }
318     } finally {
319       Closeables.close(scanner, true);
320       for (StoreFile f : readersToClose) {
321         try {
322           f.closeReader(true);
323         } catch (IOException e) {
324           LOG.warn("Exception closing " + f, e);
325         }
326       }
327       if (!finished && writer != null) {
328         abortWriter(writer);
329       }
330     }
331     assert finished : "We should have exited the method on all error paths";
332     assert writer != null : "Writer should be non-null if no error";
333     return commitWriter(writer, fd, request);
334   }
335 
336   protected abstract List<Path> commitWriter(T writer, FileDetails fd, CompactionRequest request)
337       throws IOException;
338 
339   protected abstract void abortWriter(T writer) throws IOException;
340 
341   /**
342    * Calls coprocessor, if any, to create compaction scanner - before normal scanner creation.
343    * @param request Compaction request.
344    * @param scanType Scan type.
345    * @param earliestPutTs Earliest put ts.
346    * @param scanners File scanners for compaction files.
347    * @return Scanner override by coprocessor; null if not overriding.
348    */
349   protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
350       ScanType scanType, long earliestPutTs,  List<StoreFileScanner> scanners) throws IOException {
351     return preCreateCoprocScanner(request, scanType, earliestPutTs, scanners, null);
352   }
353 
354   protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
355       final ScanType scanType, final long earliestPutTs, final List<StoreFileScanner> scanners,
356       User user) throws IOException {
357     if (store.getCoprocessorHost() == null) {
358       return null;
359     }
360     if (user == null) {
361       return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
362         earliestPutTs, request);
363     } else {
364       try {
365         return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
366           @Override
367           public InternalScanner run() throws Exception {
368             return store.getCoprocessorHost().preCompactScannerOpen(store, scanners,
369               scanType, earliestPutTs, request);
370           }
371         });
372       } catch (InterruptedException ie) {
373         InterruptedIOException iioe = new InterruptedIOException();
374         iioe.initCause(ie);
375         throw iioe;
376       }
377     }
378   }
379 
380   /**
381    * Calls coprocessor, if any, to create scanners - after normal scanner creation.
382    * @param request Compaction request.
383    * @param scanType Scan type.
384    * @param scanner The default scanner created for compaction.
385    * @return Scanner scanner to use (usually the default); null if compaction should not proceed.
386    */
387   protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
388       final ScanType scanType, final InternalScanner scanner, User user) throws IOException {
389     if (store.getCoprocessorHost() == null) {
390       return scanner;
391     }
392     if (user == null) {
393       return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
394     } else {
395       try {
396         return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
397           @Override
398           public InternalScanner run() throws Exception {
399             return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
400           }
401         });
402       } catch (InterruptedException ie) {
403         InterruptedIOException iioe = new InterruptedIOException();
404         iioe.initCause(ie);
405         throw iioe;
406       }
407     }
408   }
409 
410   /**
411    * Performs the compaction.
412    * @param fd FileDetails of cell sink writer
413    * @param scanner Where to read from.
414    * @param writer Where to write to.
415    * @param smallestReadPoint Smallest read point.
416    * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is &lt;=
417    *          smallestReadPoint
418    * @param major Is a major compaction.
419    * @return Whether compaction ended; false if it was interrupted for some reason.
420    */
421   protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
422       long smallestReadPoint, boolean cleanSeqId,
423       ThroughputController throughputController, boolean major) throws IOException {
424     long bytesWrittenProgressForCloseCheck = 0;
425     long bytesWrittenProgressForLog = 0;
426     long bytesWrittenProgressForShippedCall = 0;
427     // Since scanner.next() can return 'false' but still be delivering data,
428     // we have to use a do/while loop.
429     List<Cell> cells = new ArrayList<Cell>();
430     long closeCheckSizeLimit = HStore.getCloseCheckInterval();
431     long lastMillis = 0;
432     if (LOG.isDebugEnabled()) {
433       lastMillis = EnvironmentEdgeManager.currentTime();
434     }
435     String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
436     long now = 0;
437     boolean hasMore;
438     ScannerContext scannerContext =
439         ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
440 
441     throughputController.start(compactionName);
442     KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
443     long minFilesToCompact = Math.max(2L,
444         conf.getInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY,
445             /* old name */ conf.getInt("hbase.hstore.compactionThreshold", 3)));
446     long shippedCallSizeLimit = (long) minFilesToCompact * HConstants.DEFAULT_BLOCKSIZE;
447     try {
448       do {
449         hasMore = scanner.next(cells, scannerContext);
450         if (LOG.isDebugEnabled()) {
451           now = EnvironmentEdgeManager.currentTime();
452         }
453         // output to writer:
454         for (Cell c : cells) {
455           if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
456             CellUtil.setSequenceId(c, 0);
457           }
458           writer.append(c);
459           int len = KeyValueUtil.length(c);
460           ++progress.currentCompactedKVs;
461           progress.totalCompactedSize += len;
462           bytesWrittenProgressForShippedCall += len;
463           if (LOG.isDebugEnabled()) {
464             bytesWrittenProgressForLog += len;
465           }
466           throughputController.control(compactionName, len);
467           // check periodically to see if a system stop is requested
468           if (closeCheckSizeLimit > 0) {
469             bytesWrittenProgressForCloseCheck += len;
470             if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
471               bytesWrittenProgressForCloseCheck = 0;
472               if (!store.areWritesEnabled()) {
473                 progress.cancel();
474                 return false;
475               }
476             }
477           }
478           if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
479             // The SHARED block references, being read for compaction, will be kept in prevBlocks
480             // list(See HFileScannerImpl#prevBlocks). In case of scan flow, after each set of cells
481             // being returned to client, we will call shipped() which can clear this list. Here by
482             // we are doing the similar thing. In between the compaction (after every N cells
483             // written with collective size of 'shippedCallSizeLimit') we will call shipped which
484             // may clear prevBlocks list.
485             kvs.shipped();
486             bytesWrittenProgressForShippedCall = 0;
487           }
488         }
489         // Log the progress of long running compactions every minute if
490         // logging at DEBUG level
491         if (LOG.isDebugEnabled()) {
492           if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) {
493             LOG.debug("Compaction progress: "
494                 + compactionName
495                 + " "
496                 + progress
497                 + String.format(", rate=%.2f kB/sec", (bytesWrittenProgressForLog / 1024.0)
498                     / ((now - lastMillis) / 1000.0)) + ", throughputController is "
499                 + throughputController);
500             lastMillis = now;
501             bytesWrittenProgressForLog = 0;
502           }
503         }
504         cells.clear();
505       } while (hasMore);
506     } catch (InterruptedException e) {
507       progress.cancel();
508       throw new InterruptedIOException("Interrupted while control throughput of compacting "
509           + compactionName);
510     } finally {
511       throughputController.finish(compactionName);
512     }
513     progress.complete();
514     return true;
515   }
516 
517   /**
518    * @param store store
519    * @param scanners Store file scanners.
520    * @param scanType Scan type.
521    * @param smallestReadPoint Smallest MVCC read point.
522    * @param earliestPutTs Earliest put across all files.
523    * @return A compaction scanner.
524    */
525   protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
526       ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
527     Scan scan = new Scan();
528     scan.setMaxVersions(store.getFamily().getMaxVersions());
529     return new StoreScanner(store, store.getScanInfo(), scan, scanners,
530         scanType, smallestReadPoint, earliestPutTs);
531   }
532 
533   /**
534    * @param store The store.
535    * @param scanners Store file scanners.
536    * @param smallestReadPoint Smallest MVCC read point.
537    * @param earliestPutTs Earliest put across all files.
538    * @param dropDeletesFromRow Drop deletes starting with this row, inclusive. Can be null.
539    * @param dropDeletesToRow Drop deletes ending with this row, exclusive. Can be null.
540    * @return A compaction scanner.
541    */
542   protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
543      long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
544      byte[] dropDeletesToRow) throws IOException {
545     Scan scan = new Scan();
546     scan.setMaxVersions(store.getFamily().getMaxVersions());
547     return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
548         earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
549   }
550 }