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