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.security.User;
33
34
35
36
37 @InterfaceAudience.Private
38 public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTieredMultiFileWriter> {
39
40 private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
41
42 public DateTieredCompactor(Configuration conf, Store store) {
43 super(conf, store);
44 }
45
46 private boolean needEmptyFile(CompactionRequest request) {
47
48
49 return store.getMaxSequenceId() == StoreFile.getMaxSequenceIdInList(request.getFiles());
50 }
51
52 public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,
53 CompactionThroughputController throughputController, User user) throws IOException {
54 if (LOG.isDebugEnabled()) {
55 LOG.debug("Executing compaction with " + lowerBoundaries.size()
56 + "windows, lower boundaries: " + lowerBoundaries);
57 }
58 return compact(request, defaultScannerFactory,
59 new CellSinkFactory<DateTieredMultiFileWriter>() {
60
61 @Override
62 public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
63 boolean shouldDropBehind) throws IOException {
64 DateTieredMultiFileWriter writer = new DateTieredMultiFileWriter(lowerBoundaries,
65 needEmptyFile(request));
66 initMultiWriter(writer, scanner, fd, shouldDropBehind);
67 return writer;
68 }
69 }, throughputController, user);
70 }
71
72 @Override
73 protected List<Path> commitWriter(DateTieredMultiFileWriter writer, FileDetails fd,
74 CompactionRequest request) throws IOException {
75 return writer.commitWriters(fd.maxSeqId, request.isMajor());
76 }
77 }