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.io.InterruptedIOException;
22 import java.util.ArrayList;
23 import java.util.Collection;
24 import java.util.List;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.io.compress.Compression;
32 import org.apache.hadoop.hbase.regionserver.InternalScanner;
33 import org.apache.hadoop.hbase.regionserver.ScanType;
34 import org.apache.hadoop.hbase.regionserver.Store;
35 import org.apache.hadoop.hbase.regionserver.StoreFile;
36 import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
37 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
38 import org.apache.hadoop.hbase.regionserver.StoreScanner;
39 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
40 import org.apache.hadoop.hbase.security.User;
41 import org.apache.hadoop.hbase.util.Bytes;
42
43
44
45
46
47 @InterfaceAudience.Private
48 public class StripeCompactor extends Compactor {
49 private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
50 public StripeCompactor(Configuration conf, Store store) {
51 super(conf, store);
52 }
53
54 public List<Path> compact(CompactionRequest request, List<byte[]> targetBoundaries,
55 byte[] majorRangeFromRow, byte[] majorRangeToRow,
56 CompactionThroughputController throughputController) throws IOException {
57 return compact(request, targetBoundaries, majorRangeFromRow, majorRangeToRow,
58 throughputController, null);
59 }
60 public List<Path> compact(CompactionRequest request, List<byte[]> targetBoundaries,
61 byte[] majorRangeFromRow, byte[] majorRangeToRow,
62 CompactionThroughputController throughputController, User user) throws IOException {
63 if (LOG.isDebugEnabled()) {
64 StringBuilder sb = new StringBuilder();
65 sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:");
66 for (byte[] tb : targetBoundaries) {
67 sb.append(" [").append(Bytes.toString(tb)).append("]");
68 }
69 LOG.debug(sb.toString());
70 }
71 StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter(
72 targetBoundaries, majorRangeFromRow, majorRangeToRow);
73 return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
74 throughputController, user);
75 }
76
77 public List<Path> compact(CompactionRequest request, int targetCount, long targetSize,
78 byte[] left, byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
79 CompactionThroughputController throughputController) throws IOException {
80 return compact(request, targetCount, targetSize, left, right, majorRangeFromRow,
81 majorRangeToRow, throughputController, null);
82 }
83
84 public List<Path> compact(CompactionRequest request, int targetCount, long targetSize,
85 byte[] left, byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
86 CompactionThroughputController throughputController, User user) throws IOException {
87 if (LOG.isDebugEnabled()) {
88 LOG.debug("Executing compaction with " + targetSize
89 + " target file size, no more than " + targetCount + " files, in ["
90 + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range");
91 }
92 StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(
93 targetCount, targetSize, left, right);
94 return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
95 throughputController, user);
96 }
97
98 private List<Path> compactInternal(StripeMultiFileWriter mw, final CompactionRequest request,
99 byte[] majorRangeFromRow, byte[] majorRangeToRow,
100 CompactionThroughputController throughputController, User user) throws IOException {
101 final Collection<StoreFile> filesToCompact = request.getFiles();
102 final FileDetails fd = getFileDetails(filesToCompact, request.isMajor());
103 this.progress = new CompactionProgress(fd.maxKeyCount);
104
105 long smallestReadPoint = getSmallestReadPoint();
106 List<StoreFileScanner> scanners = createFileScanners(filesToCompact,
107 smallestReadPoint, store.throttleCompaction(request.getSize()));
108
109 boolean finished = false;
110 InternalScanner scanner = null;
111 try {
112
113 ScanType coprocScanType = ScanType.COMPACT_RETAIN_DELETES;
114 scanner = preCreateCoprocScanner(request, coprocScanType, fd.earliestPutTs, scanners, user);
115 if (scanner == null) {
116 scanner = (majorRangeFromRow == null)
117 ? createScanner(store, scanners,
118 ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint, fd.earliestPutTs)
119 : createScanner(store, scanners,
120 smallestReadPoint, fd.earliestPutTs, majorRangeFromRow, majorRangeToRow);
121 }
122 scanner = postCreateCoprocScanner(request, coprocScanType, scanner, user);
123 if (scanner == null) {
124
125 return new ArrayList<Path>();
126 }
127
128
129 final boolean needMvcc = fd.maxMVCCReadpoint >= smallestReadPoint;
130 final Compression.Algorithm compression = store.getFamily().getCompactionCompression();
131 StripeMultiFileWriter.WriterFactory factory = new StripeMultiFileWriter.WriterFactory() {
132 @Override
133 public Writer createWriter() throws IOException {
134 return store.createWriterInTmp(
135 fd.maxKeyCount, compression, true, needMvcc, fd.maxTagsLength > 0,
136 store.throttleCompaction(request.getSize()));
137 }
138 };
139
140
141
142 StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
143 mw.init(storeScanner, factory, store.getComparator());
144 finished = performCompaction(scanner, mw, smallestReadPoint, throughputController);
145 if (!finished) {
146 throw new InterruptedIOException( "Aborting compaction of store " + store +
147 " in region " + store.getRegionInfo().getRegionNameAsString() +
148 " because it was interrupted.");
149 }
150 } finally {
151 if (scanner != null) {
152 try {
153 scanner.close();
154 } catch (Throwable t) {
155
156 LOG.error("Failed to close scanner after compaction.", t);
157 }
158 }
159 if (!finished) {
160 for (Path leftoverFile : mw.abortWriters()) {
161 try {
162 store.getFileSystem().delete(leftoverFile, false);
163 } catch (Exception ex) {
164 LOG.error("Failed to delete the leftover file after an unfinished compaction.", ex);
165 }
166 }
167 }
168 }
169
170 assert finished : "We should have exited the method on all error paths";
171 List<Path> newFiles = mw.commitWriters(fd.maxSeqId, request.isMajor());
172 assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
173 return newFiles;
174 }
175 }