1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.compactions;
19
20 import java.io.IOException;
21 import java.util.List;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
29 import org.apache.hadoop.hbase.regionserver.InternalScanner;
30 import org.apache.hadoop.hbase.regionserver.Store;
31 import org.apache.hadoop.hbase.regionserver.StoreFile;
32 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
33 import org.apache.hadoop.hbase.security.User;
34
35
36
37
38 @InterfaceAudience.Private
39 public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTieredMultiFileWriter> {
40
41 private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
42
43 public DateTieredCompactor(Configuration conf, Store store) {
44 super(conf, store);
45 }
46
47 private boolean needEmptyFile(CompactionRequest request) {
48
49
50 return StoreFile.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
51 }
52
53 public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,
54 ThroughputController throughputController, User user) throws IOException {
55 if (LOG.isDebugEnabled()) {
56 LOG.debug("Executing compaction with " + lowerBoundaries.size()
57 + "windows, lower boundaries: " + lowerBoundaries);
58 }
59
60 return compact(request, defaultScannerFactory,
61 new CellSinkFactory<DateTieredMultiFileWriter>() {
62
63 @Override
64 public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
65 boolean shouldDropBehind) throws IOException {
66 DateTieredMultiFileWriter writer = new DateTieredMultiFileWriter(lowerBoundaries,
67 needEmptyFile(request));
68 initMultiWriter(writer, scanner, fd, shouldDropBehind);
69 return writer;
70 }
71 }, throughputController, user);
72 }
73
74 @Override
75 protected List<Path> commitWriter(DateTieredMultiFileWriter writer, FileDetails fd,
76 CompactionRequest request) throws IOException {
77 return writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles());
78 }
79 }