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 import java.util.Map;
26 import java.util.concurrent.atomic.AtomicInteger;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.Cell;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.KeyValue;
34 import org.apache.hadoop.hbase.KeyValueUtil;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.client.Scan;
37 import org.apache.hadoop.hbase.io.compress.Compression;
38 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
39 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
40 import org.apache.hadoop.hbase.regionserver.HStore;
41 import org.apache.hadoop.hbase.regionserver.InternalScanner;
42 import org.apache.hadoop.hbase.regionserver.ScanType;
43 import org.apache.hadoop.hbase.regionserver.Store;
44 import org.apache.hadoop.hbase.regionserver.StoreFile;
45 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
46 import org.apache.hadoop.hbase.regionserver.StoreScanner;
47 import org.apache.hadoop.hbase.util.Bytes;
48 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
49 import org.apache.hadoop.util.StringUtils;
50
51
52
53
54
55 @InterfaceAudience.Private
56 public abstract class Compactor {
57 private static final Log LOG = LogFactory.getLog(Compactor.class);
58 protected CompactionProgress progress;
59 protected Configuration conf;
60 protected Store store;
61
62 private int compactionKVMax;
63 protected Compression.Algorithm compactionCompression;
64
65
66 Compactor(final Configuration conf, final Store store) {
67 this.conf = conf;
68 this.store = store;
69 this.compactionKVMax =
70 this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
71 this.compactionCompression = (this.store.getFamily() == null) ?
72 Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
73 }
74
75
76
77
78 public interface CellSink {
79 void append(KeyValue kv) throws IOException;
80 }
81
82 public CompactionProgress getProgress() {
83 return this.progress;
84 }
85
86
87 protected static class FileDetails {
88
89 public long maxKeyCount = 0;
90
91 public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
92
93 public long maxSeqId = 0;
94
95 public long maxMVCCReadpoint = 0;
96
97 public int maxTagsLength = 0;
98 }
99
100
101
102
103
104
105
106 protected FileDetails getFileDetails(
107 Collection<StoreFile> filesToCompact, boolean calculatePutTs) throws IOException {
108 FileDetails fd = new FileDetails();
109
110 for (StoreFile file : filesToCompact) {
111 long seqNum = file.getMaxSequenceId();
112 fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
113 StoreFile.Reader r = file.getReader();
114 if (r == null) {
115 LOG.warn("Null reader for " + file.getPath());
116 continue;
117 }
118
119
120
121 long keyCount = r.getEntries();
122 fd.maxKeyCount += keyCount;
123
124 Map<byte[], byte[]> fileInfo = r.loadFileInfo();
125 byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
126 if (tmp != null) {
127 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
128 }
129 tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
130 if (tmp != null) {
131 fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
132 }
133
134
135 long earliestPutTs = 0;
136 if (calculatePutTs) {
137 tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
138 if (tmp == null) {
139
140
141 fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
142 } else {
143 earliestPutTs = Bytes.toLong(tmp);
144 fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
145 }
146 }
147 if (LOG.isDebugEnabled()) {
148 LOG.debug("Compacting " + file +
149 ", keycount=" + keyCount +
150 ", bloomtype=" + r.getBloomFilterType().toString() +
151 ", size=" + StringUtils.humanReadableInt(r.length()) +
152 ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
153 ", seqNum=" + seqNum +
154 (calculatePutTs ? ", earliestPutTs=" + earliestPutTs: ""));
155 }
156 }
157 return fd;
158 }
159
160
161
162
163
164
165 protected List<StoreFileScanner> createFileScanners(
166 final Collection<StoreFile> filesToCompact, long smallestReadPoint) throws IOException {
167 return StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true,
168 smallestReadPoint);
169 }
170
171 protected long getSmallestReadPoint() {
172 return store.getSmallestReadPoint();
173 }
174
175
176
177
178
179
180
181
182
183 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
184 ScanType scanType, long earliestPutTs, List<StoreFileScanner> scanners) throws IOException {
185 if (store.getCoprocessorHost() == null) return null;
186 return store.getCoprocessorHost()
187 .preCompactScannerOpen(store, scanners, scanType, earliestPutTs, request);
188 }
189
190
191
192
193
194
195
196
197 protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
198 ScanType scanType, InternalScanner scanner) throws IOException {
199 if (store.getCoprocessorHost() == null) return scanner;
200 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
201 }
202
203
204
205
206
207 private static final AtomicInteger NAME_COUNTER = new AtomicInteger(0);
208
209 private String generateCompactionName() {
210 int counter;
211 for (;;) {
212 counter = NAME_COUNTER.get();
213 int next = counter == Integer.MAX_VALUE ? 0 : counter + 1;
214 if (NAME_COUNTER.compareAndSet(counter, next)) {
215 break;
216 }
217 }
218 return store.getRegionInfo().getRegionNameAsString() + "#"
219 + store.getFamily().getNameAsString() + "#" + counter;
220 }
221
222
223
224
225
226
227
228 protected boolean performCompaction(InternalScanner scanner, CellSink writer,
229 long smallestReadPoint, CompactionThroughputController throughputController)
230 throws IOException {
231 long bytesWritten = 0;
232 long bytesWrittenProgress = 0;
233
234
235 List<Cell> kvs = new ArrayList<Cell>();
236 long closeCheckInterval = HStore.getCloseCheckInterval();
237 long lastMillis = 0;
238 if (LOG.isDebugEnabled()) {
239 lastMillis = EnvironmentEdgeManager.currentTimeMillis();
240 }
241 String compactionName = generateCompactionName();
242 long now = 0;
243 boolean hasMore;
244 throughputController.start(compactionName);
245 try {
246 do {
247 hasMore = scanner.next(kvs, compactionKVMax);
248 if (LOG.isDebugEnabled()) {
249 now = EnvironmentEdgeManager.currentTimeMillis();
250 }
251
252 for (Cell c : kvs) {
253 KeyValue kv = KeyValueUtil.ensureKeyValue(c);
254 if (kv.getMvccVersion() <= smallestReadPoint) {
255 kv.setMvccVersion(0);
256 }
257 writer.append(kv);
258 int len = kv.getLength();
259 ++progress.currentCompactedKVs;
260 progress.totalCompactedSize += len;
261 if (LOG.isDebugEnabled()) {
262 bytesWrittenProgress += len;
263 }
264 throughputController.control(compactionName, len);
265
266
267 if (closeCheckInterval > 0) {
268 bytesWritten += len;
269 if (bytesWritten > closeCheckInterval) {
270 bytesWritten = 0;
271 if (!store.areWritesEnabled()) {
272 progress.cancel();
273 return false;
274 }
275 }
276 }
277 }
278
279
280 if (LOG.isDebugEnabled()) {
281 if ((now - lastMillis) >= 60 * 1000) {
282 LOG.debug("Compaction progress: "
283 + compactionName
284 + " "
285 + progress
286 + String.format(", rate=%.2f kB/sec", (bytesWrittenProgress / 1024.0)
287 / ((now - lastMillis) / 1000.0)) + ", throughputController is "
288 + throughputController);
289 lastMillis = now;
290 bytesWrittenProgress = 0;
291 }
292 }
293 kvs.clear();
294 } while (hasMore);
295 } catch (InterruptedException e) {
296 throw new InterruptedIOException("Interrupted while control throughput of compacting "
297 + compactionName);
298 } finally {
299 throughputController.finish(compactionName);
300 progress.complete();
301 }
302 return true;
303 }
304
305
306
307
308
309
310
311
312
313 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
314 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
315 Scan scan = new Scan();
316 scan.setMaxVersions(store.getFamily().getMaxVersions());
317 return new StoreScanner(store, store.getScanInfo(), scan, scanners,
318 scanType, smallestReadPoint, earliestPutTs);
319 }
320
321
322
323
324
325
326
327
328
329
330 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
331 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
332 byte[] dropDeletesToRow) throws IOException {
333 Scan scan = new Scan();
334 scan.setMaxVersions(store.getFamily().getMaxVersions());
335 return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
336 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
337 }
338 }