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.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24 import java.util.Map;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.KeyValue;
33 import org.apache.hadoop.hbase.client.Scan;
34 import org.apache.hadoop.hbase.io.CellOutputStream;
35 import org.apache.hadoop.hbase.io.compress.Compression;
36 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
37 import org.apache.hadoop.hbase.regionserver.HStore;
38 import org.apache.hadoop.hbase.regionserver.InternalScanner;
39 import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
40 import org.apache.hadoop.hbase.regionserver.ScanType;
41 import org.apache.hadoop.hbase.regionserver.Store;
42 import org.apache.hadoop.hbase.regionserver.StoreFile;
43 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
44 import org.apache.hadoop.hbase.regionserver.StoreScanner;
45 import org.apache.hadoop.hbase.util.Bytes;
46 import org.apache.hadoop.util.StringUtils;
47
48
49
50
51
52 @InterfaceAudience.Private
53 public abstract class Compactor {
54 private static final Log LOG = LogFactory.getLog(Compactor.class);
55 protected CompactionProgress progress;
56 protected Configuration conf;
57 protected Store store;
58
59 private int compactionKVMax;
60 protected Compression.Algorithm compactionCompression;
61
62
63 Compactor(final Configuration conf, final Store store) {
64 this.conf = conf;
65 this.store = store;
66 this.compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
67 this.compactionCompression = (this.store.getFamily() == null) ?
68 Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
69 }
70
71
72
73
74 public interface CellSink {
75 void append(KeyValue kv) throws IOException;
76 }
77
78
79
80
81
82
83
84
85 public abstract List<Path> compact(final CompactionRequest request) throws IOException;
86
87
88
89
90
91
92
93
94
95
96
97 public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
98 throws IOException {
99 CompactionRequest cr = new CompactionRequest(filesToCompact);
100 cr.setIsMajor(isMajor);
101 return this.compact(cr);
102 }
103
104 public CompactionProgress getProgress() {
105 return this.progress;
106 }
107
108
109 protected static class FileDetails {
110
111 public long maxKeyCount = 0;
112
113 public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
114
115 public long maxSeqId = 0;
116
117 public long maxMVCCReadpoint = 0;
118 }
119
120 protected FileDetails getFileDetails(
121 Collection<StoreFile> filesToCompact, boolean calculatePutTs) throws IOException {
122 FileDetails fd = new FileDetails();
123
124 for (StoreFile file : filesToCompact) {
125 long seqNum = file.getMaxSequenceId();
126 fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
127 StoreFile.Reader r = file.getReader();
128 if (r == null) {
129 LOG.warn("Null reader for " + file.getPath());
130 continue;
131 }
132
133
134 long keyCount = (r.getBloomFilterType() == store.getFamily().getBloomFilterType())
135 ? r.getFilterEntries() : r.getEntries();
136 fd.maxKeyCount += keyCount;
137
138 Map<byte[], byte[]> fileInfo = r.loadFileInfo();
139 byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
140 if (tmp != null) {
141 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
142 }
143
144
145 long earliestPutTs = 0;
146 if (calculatePutTs) {
147 tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
148 if (tmp == null) {
149
150
151 fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
152 } else {
153 earliestPutTs = Bytes.toLong(tmp);
154 fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
155 }
156 }
157 if (LOG.isDebugEnabled()) {
158 LOG.debug("Compacting " + file +
159 ", keycount=" + keyCount +
160 ", bloomtype=" + r.getBloomFilterType().toString() +
161 ", size=" + StringUtils.humanReadableInt(r.length()) +
162 ", encoding=" + r.getHFileReader().getEncodingOnDisk() +
163 ", seqNum=" + seqNum +
164 (calculatePutTs ? ", earliestPutTs=" + earliestPutTs: ""));
165 }
166 }
167 return fd;
168 }
169
170 protected List<StoreFileScanner> createFileScanners(
171 final Collection<StoreFile> filesToCompact) throws IOException {
172 return StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true);
173 }
174
175 protected long setSmallestReadPoint() {
176 long smallestReadPoint = store.getSmallestReadPoint();
177 MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
178 return smallestReadPoint;
179 }
180
181 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
182 ScanType scanType, long earliestPutTs, List<StoreFileScanner> scanners) throws IOException {
183 if (store.getCoprocessorHost() == null) return null;
184 return store.getCoprocessorHost()
185 .preCompactScannerOpen(store, scanners, scanType, earliestPutTs, request);
186 }
187
188 protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
189 ScanType scanType, InternalScanner scanner) throws IOException {
190 if (store.getCoprocessorHost() == null) return scanner;
191 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
192 }
193
194 @SuppressWarnings("deprecation")
195 protected boolean performCompaction(InternalScanner scanner,
196 CellSink writer, long smallestReadPoint) throws IOException {
197 int bytesWritten = 0;
198
199
200 List<KeyValue> kvs = new ArrayList<KeyValue>();
201
202 int closeCheckInterval = HStore.getCloseCheckInterval();
203 boolean hasMore;
204 do {
205 hasMore = scanner.next(kvs, compactionKVMax);
206
207 for (KeyValue kv : kvs) {
208 if (kv.getMvccVersion() <= smallestReadPoint) {
209 kv.setMvccVersion(0);
210 }
211 writer.append(kv);
212 ++progress.currentCompactedKVs;
213
214
215 if (closeCheckInterval > 0) {
216 bytesWritten += kv.getLength();
217 if (bytesWritten > closeCheckInterval) {
218 bytesWritten = 0;
219 if (!store.areWritesEnabled()) {
220 progress.cancel();
221 return false;
222 }
223 }
224 }
225 }
226 kvs.clear();
227 } while (hasMore);
228 progress.complete();
229 return true;
230 }
231
232 protected void abortWriter(final StoreFile.Writer writer) throws IOException {
233 writer.close();
234 store.getFileSystem().delete(writer.getPath(), false);
235 }
236
237
238
239
240
241
242
243
244 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
245 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
246 Scan scan = new Scan();
247 scan.setMaxVersions(store.getFamily().getMaxVersions());
248 return new StoreScanner(store, store.getScanInfo(), scan, scanners,
249 scanType, smallestReadPoint, earliestPutTs);
250 }
251 }