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.InternalScanner;
29 import org.apache.hadoop.hbase.regionserver.ScanType;
30 import org.apache.hadoop.hbase.regionserver.Store;
31 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
32 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
33 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
34 import org.apache.hadoop.hbase.security.User;
35 import org.apache.hadoop.hbase.util.Bytes;
36
37
38
39
40
41 @InterfaceAudience.Private
42 public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> {
43 private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
44
45 public StripeCompactor(Configuration conf, Store store) {
46 super(conf, store);
47 }
48
49 private final class StripeInternalScannerFactory implements InternalScannerFactory {
50
51 private final byte[] majorRangeFromRow;
52
53 private final byte[] majorRangeToRow;
54
55 public StripeInternalScannerFactory(byte[] majorRangeFromRow, byte[] majorRangeToRow) {
56 this.majorRangeFromRow = majorRangeFromRow;
57 this.majorRangeToRow = majorRangeToRow;
58 }
59
60 @Override
61 public ScanType getScanType(CompactionRequest request) {
62
63
64
65 return ScanType.COMPACT_RETAIN_DELETES;
66 }
67
68 @Override
69 public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
70 FileDetails fd, long smallestReadPoint) throws IOException {
71 return (majorRangeFromRow == null)
72 ? StripeCompactor.this.createScanner(store, scanners, scanType, smallestReadPoint,
73 fd.earliestPutTs)
74 : StripeCompactor.this.createScanner(store, scanners, smallestReadPoint, fd.earliestPutTs,
75 majorRangeFromRow, majorRangeToRow);
76 }
77 }
78
79 public List<Path> compact(CompactionRequest request, final List<byte[]> targetBoundaries,
80 final byte[] majorRangeFromRow, final byte[] majorRangeToRow,
81 ThroughputController throughputController, User user) throws IOException {
82 if (LOG.isDebugEnabled()) {
83 StringBuilder sb = new StringBuilder();
84 sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:");
85 for (byte[] tb : targetBoundaries) {
86 sb.append(" [").append(Bytes.toString(tb)).append("]");
87 }
88 LOG.debug(sb.toString());
89 }
90 return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow),
91 new CellSinkFactory<StripeMultiFileWriter>() {
92
93 @Override
94 public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
95 boolean shouldDropBehind) throws IOException {
96 StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter(
97 store.getComparator(), targetBoundaries, majorRangeFromRow, majorRangeToRow);
98 initMultiWriter(writer, scanner, fd, shouldDropBehind);
99 return writer;
100 }
101 }, throughputController, user);
102 }
103
104 public List<Path> compact(CompactionRequest request, final int targetCount, final long targetSize,
105 final byte[] left, final byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
106 ThroughputController throughputController, User user) throws IOException {
107 if (LOG.isDebugEnabled()) {
108 LOG.debug(
109 "Executing compaction with " + targetSize + " target file size, no more than " + targetCount
110 + " files, in [" + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range");
111 }
112 return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow),
113 new CellSinkFactory<StripeMultiFileWriter>() {
114
115 @Override
116 public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
117 boolean shouldDropBehind) throws IOException {
118 StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(
119 store.getComparator(), targetCount, targetSize, left, right);
120 initMultiWriter(writer, scanner, fd, shouldDropBehind);
121 return writer;
122 }
123 }, throughputController, user);
124 }
125
126 @Override
127 protected List<Path> commitWriter(StripeMultiFileWriter writer, FileDetails fd,
128 CompactionRequest request) throws IOException {
129 List<Path> newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor(), request.getFiles());
130 assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
131 return newFiles;
132 }
133 }