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