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 com.google.common.io.Closeables;
21
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.security.PrivilegedExceptionAction;
25 import java.util.ArrayList;
26 import java.util.Collection;
27 import java.util.Collections;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.concurrent.atomic.AtomicInteger;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.Cell;
37 import org.apache.hadoop.hbase.CellUtil;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.KeyValueUtil;
40 import org.apache.hadoop.hbase.classification.InterfaceAudience;
41 import org.apache.hadoop.hbase.client.Scan;
42 import org.apache.hadoop.hbase.io.compress.Compression;
43 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
44 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
45 import org.apache.hadoop.hbase.regionserver.HStore;
46 import org.apache.hadoop.hbase.regionserver.InternalScanner;
47 import org.apache.hadoop.hbase.regionserver.ScanType;
48 import org.apache.hadoop.hbase.regionserver.ScannerContext;
49 import org.apache.hadoop.hbase.regionserver.Store;
50 import org.apache.hadoop.hbase.regionserver.StoreFile;
51 import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
52 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
53 import org.apache.hadoop.hbase.regionserver.StoreScanner;
54 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
55 import org.apache.hadoop.hbase.regionserver.compactions.Compactor.CellSink;
56 import org.apache.hadoop.hbase.security.User;
57 import org.apache.hadoop.hbase.util.Bytes;
58 import org.apache.hadoop.hbase.util.Writables;
59 import org.apache.hadoop.util.StringUtils;
60 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
61 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
62
63
64
65
66
67 @InterfaceAudience.Private
68 public abstract class Compactor<T extends CellSink> {
69 private static final Log LOG = LogFactory.getLog(Compactor.class);
70
71 protected volatile CompactionProgress progress;
72
73 protected final Configuration conf;
74 protected final Store store;
75
76 protected final int compactionKVMax;
77 protected final Compression.Algorithm compactionCompression;
78
79
80 protected final int keepSeqIdPeriod;
81
82
83 Compactor(final Configuration conf, final Store store) {
84 this.conf = conf;
85 this.store = store;
86 this.compactionKVMax =
87 this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
88 this.compactionCompression = (this.store.getFamily() == null) ?
89 Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
90 this.keepSeqIdPeriod = Math.max(this.conf.getInt(HConstants.KEEP_SEQID_PERIOD,
91 HConstants.MIN_KEEP_SEQID_PERIOD), HConstants.MIN_KEEP_SEQID_PERIOD);
92 }
93
94 public interface CellSink {
95 void append(Cell cell) throws IOException;
96 }
97
98 protected interface CellSinkFactory<S> {
99 S createWriter(InternalScanner scanner, FileDetails fd, boolean shouldDropBehind)
100 throws IOException;
101 }
102
103 public CompactionProgress getProgress() {
104 return this.progress;
105 }
106
107
108 protected static class FileDetails {
109
110 public long maxKeyCount = 0;
111
112 public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
113
114 public long latestPutTs = HConstants.LATEST_TIMESTAMP;
115
116 public long maxSeqId = 0;
117
118 public long maxMVCCReadpoint = 0;
119
120 public int maxTagsLength = 0;
121
122 public long minSeqIdToKeep = 0;
123 }
124
125
126
127
128
129
130
131 protected FileDetails getFileDetails(
132 Collection<StoreFile> filesToCompact, boolean allFiles) throws IOException {
133 FileDetails fd = new FileDetails();
134 long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() -
135 (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);
136
137 for (StoreFile file : filesToCompact) {
138 if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
139
140
141 if(fd.minSeqIdToKeep < file.getMaxMemstoreTS()) {
142 fd.minSeqIdToKeep = file.getMaxMemstoreTS();
143 }
144 }
145 long seqNum = file.getMaxSequenceId();
146 fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
147 StoreFile.Reader r = file.getReader();
148 if (r == null) {
149 LOG.warn("Null reader for " + file.getPath());
150 continue;
151 }
152
153
154
155 long keyCount = r.getEntries();
156 fd.maxKeyCount += keyCount;
157
158 Map<byte[], byte[]> fileInfo = r.loadFileInfo();
159 byte[] tmp = null;
160
161
162 if (r.isBulkLoaded()) {
163 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, r.getSequenceID());
164 }
165 else {
166 tmp = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
167 if (tmp != null) {
168 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
169 }
170 }
171 tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
172 if (tmp != null) {
173 fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
174 }
175
176
177 long earliestPutTs = 0;
178 if (allFiles) {
179 tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
180 if (tmp == null) {
181
182
183 fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
184 } else {
185 earliestPutTs = Bytes.toLong(tmp);
186 fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
187 }
188 }
189 tmp = fileInfo.get(StoreFile.TIMERANGE_KEY);
190 TimeRangeTracker trt = new TimeRangeTracker();
191 if (tmp == null) {
192 fd.latestPutTs = HConstants.LATEST_TIMESTAMP;
193 } else {
194 Writables.copyWritable(tmp, trt);
195 fd.latestPutTs = trt.getMaximumTimestamp();
196 }
197 if (LOG.isDebugEnabled()) {
198 LOG.debug("Compacting " + file +
199 ", keycount=" + keyCount +
200 ", bloomtype=" + r.getBloomFilterType().toString() +
201 ", size=" + TraditionalBinaryPrefix.long2String(r.length(), "", 1) +
202 ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
203 ", seqNum=" + seqNum +
204 (allFiles ? ", earliestPutTs=" + earliestPutTs: ""));
205 }
206 }
207 return fd;
208 }
209
210
211
212
213
214
215 protected List<StoreFileScanner> createFileScanners(
216 final Collection<StoreFile> filesToCompact, long smallestReadPoint) throws IOException {
217 return StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true,
218 smallestReadPoint);
219 }
220
221 protected long getSmallestReadPoint() {
222 return store.getSmallestReadPoint();
223 }
224
225 protected interface InternalScannerFactory {
226
227 ScanType getScanType(CompactionRequest request);
228
229 InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
230 FileDetails fd, long smallestReadPoint) throws IOException;
231 }
232
233 protected final InternalScannerFactory defaultScannerFactory = new InternalScannerFactory() {
234
235 @Override
236 public ScanType getScanType(CompactionRequest request) {
237 return request.isAllFiles() ? ScanType.COMPACT_DROP_DELETES
238 : ScanType.COMPACT_RETAIN_DELETES;
239 }
240
241 @Override
242 public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
243 FileDetails fd, long smallestReadPoint) throws IOException {
244 return Compactor.this.createScanner(store, scanners, scanType, smallestReadPoint,
245 fd.earliestPutTs);
246 }
247 };
248
249
250
251
252
253
254
255 protected Writer createTmpWriter(FileDetails fd, boolean shouldDropBehind) throws IOException {
256
257
258 return store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression,
259
260
261
262 }
263
264 protected List<Path> compact(final CompactionRequest request,
265 InternalScannerFactory scannerFactory, CellSinkFactory<T> sinkFactory,
266 CompactionThroughputController throughputController, User user) throws IOException {
267 FileDetails fd = getFileDetails(request.getFiles(), request.isAllFiles());
268 this.progress = new CompactionProgress(fd.maxKeyCount);
269
270
271 long smallestReadPoint = getSmallestReadPoint();
272
273 List<StoreFileScanner> scanners;
274 Collection<StoreFile> readersToClose;
275 T writer = null;
276 if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) {
277
278
279 readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
280 for (StoreFile f : request.getFiles()) {
281 readersToClose.add(f.cloneForReader());
282 }
283 scanners = createFileScanners(readersToClose, smallestReadPoint);
284
285 } else {
286 readersToClose = Collections.emptyList();
287 scanners = createFileScanners(request.getFiles(), smallestReadPoint);
288
289 }
290 InternalScanner scanner = null;
291 boolean finished = false;
292 try {
293
294 ScanType scanType = scannerFactory.getScanType(request);
295 scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners, user);
296 if (scanner == null) {
297 scanner = scannerFactory.createScanner(scanners, scanType, fd, smallestReadPoint);
298 }
299 scanner = postCreateCoprocScanner(request, scanType, scanner, user);
300 if (scanner == null) {
301
302 return new ArrayList<Path>();
303 }
304 boolean cleanSeqId = false;
305 if (fd.minSeqIdToKeep > 0) {
306 smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
307 cleanSeqId = true;
308 }
309 writer = sinkFactory.createWriter(scanner, fd, store.throttleCompaction(request.getSize()));
310 finished =
311 performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
312 throughputController, request.isAllFiles());
313 if (!finished) {
314 throw new InterruptedIOException("Aborting compaction of store " + store + " in region "
315 + store.getRegionInfo().getRegionNameAsString() + " because it was interrupted.");
316 }
317 } finally {
318 Closeables.close(scanner, true);
319 for (StoreFile f : readersToClose) {
320 try {
321 f.closeReader(true);
322 } catch (IOException e) {
323 LOG.warn("Exception closing " + f, e);
324 }
325 }
326 if (!finished && writer != null) {
327 abortWriter(writer);
328 }
329 }
330 assert finished : "We should have exited the method on all error paths";
331 assert writer != null : "Writer should be non-null if no error";
332 return commitWriter(writer, fd, request);
333 }
334
335 protected abstract List<Path> commitWriter(T writer, FileDetails fd, CompactionRequest request)
336 throws IOException;
337
338 protected abstract void abortWriter(T writer) throws IOException;
339
340
341
342
343
344
345
346
347
348 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
349 ScanType scanType, long earliestPutTs, List<StoreFileScanner> scanners) throws IOException {
350 return preCreateCoprocScanner(request, scanType, earliestPutTs, scanners, null);
351 }
352
353 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
354 final ScanType scanType, final long earliestPutTs, final List<StoreFileScanner> scanners,
355 User user) throws IOException {
356 if (store.getCoprocessorHost() == null) return null;
357 if (user == null) {
358 return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
359 earliestPutTs, request);
360 } else {
361 try {
362 return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
363 @Override
364 public InternalScanner run() throws Exception {
365 return store.getCoprocessorHost().preCompactScannerOpen(store, scanners,
366 scanType, earliestPutTs, request);
367 }
368 });
369 } catch (InterruptedException ie) {
370 InterruptedIOException iioe = new InterruptedIOException();
371 iioe.initCause(ie);
372 throw iioe;
373 }
374 }
375 }
376
377
378
379
380
381
382
383
384 protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
385 final ScanType scanType, final InternalScanner scanner, User user) throws IOException {
386 if (store.getCoprocessorHost() == null) {
387 return scanner;
388 }
389 if (user == null) {
390 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
391 } else {
392 try {
393 return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
394 @Override
395 public InternalScanner run() throws Exception {
396 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
397 }
398 });
399 } catch (InterruptedException ie) {
400 InterruptedIOException iioe = new InterruptedIOException();
401 iioe.initCause(ie);
402 throw iioe;
403 }
404 }
405 }
406
407
408
409
410
411 private static final AtomicInteger NAME_COUNTER = new AtomicInteger(0);
412
413 private String generateCompactionName() {
414 int counter;
415 for (;;) {
416 counter = NAME_COUNTER.get();
417 int next = counter == Integer.MAX_VALUE ? 0 : counter + 1;
418 if (NAME_COUNTER.compareAndSet(counter, next)) {
419 break;
420 }
421 }
422 return store.getRegionInfo().getRegionNameAsString() + "#"
423 + store.getFamily().getNameAsString() + "#" + counter;
424 }
425
426
427
428
429
430
431
432
433
434
435 protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
436 long smallestReadPoint, boolean cleanSeqId,
437 CompactionThroughputController throughputController, boolean major) throws IOException {
438 long bytesWritten = 0;
439 long bytesWrittenProgress = 0;
440
441
442 List<Cell> cells = new ArrayList<Cell>();
443 long closeCheckInterval = HStore.getCloseCheckInterval();
444 long lastMillis = 0;
445 if (LOG.isDebugEnabled()) {
446 lastMillis = EnvironmentEdgeManager.currentTime();
447 }
448 String compactionName = generateCompactionName();
449 long now = 0;
450 boolean hasMore;
451 ScannerContext scannerContext =
452 ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
453
454 throughputController.start(compactionName);
455 try {
456 do {
457 hasMore = scanner.next(cells, scannerContext);
458 if (LOG.isDebugEnabled()) {
459 now = EnvironmentEdgeManager.currentTime();
460 }
461
462 for (Cell c : cells) {
463 if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
464 CellUtil.setSequenceId(c, 0);
465 }
466 writer.append(c);
467 int len = KeyValueUtil.length(c);
468 ++progress.currentCompactedKVs;
469 progress.totalCompactedSize += len;
470 if (LOG.isDebugEnabled()) {
471 bytesWrittenProgress += len;
472 }
473 throughputController.control(compactionName, len);
474
475 if (closeCheckInterval > 0) {
476 bytesWritten += len;
477 if (bytesWritten > closeCheckInterval) {
478 bytesWritten = 0;
479 if (!store.areWritesEnabled()) {
480 progress.cancel();
481 return false;
482 }
483 }
484 }
485 }
486
487
488 if (LOG.isDebugEnabled()) {
489 if ((now - lastMillis) >= 60 * 1000) {
490 LOG.debug("Compaction progress: "
491 + compactionName
492 + " "
493 + progress
494 + String.format(", rate=%.2f kB/sec", (bytesWrittenProgress / 1024.0)
495 / ((now - lastMillis) / 1000.0)) + ", throughputController is "
496 + throughputController);
497 lastMillis = now;
498 bytesWrittenProgress = 0;
499 }
500 }
501 cells.clear();
502 } while (hasMore);
503 } catch (InterruptedException e) {
504 progress.cancel();
505 throw new InterruptedIOException("Interrupted while control throughput of compacting "
506 + compactionName);
507 } finally {
508 throughputController.finish(compactionName);
509 }
510 progress.complete();
511 return true;
512 }
513
514
515
516
517
518
519
520
521
522 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
523 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
524 Scan scan = new Scan();
525 scan.setMaxVersions(store.getFamily().getMaxVersions());
526 return new StoreScanner(store, store.getScanInfo(), scan, scanners,
527 scanType, smallestReadPoint, earliestPutTs);
528 }
529
530
531
532
533
534
535
536
537
538
539 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
540 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
541 byte[] dropDeletesToRow) throws IOException {
542 Scan scan = new Scan();
543 scan.setMaxVersions(store.getFamily().getMaxVersions());
544 return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
545 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
546 }
547
548
549
550
551
552
553
554
555 protected void appendMetadataAndCloseWriter(StoreFile.Writer writer, FileDetails fd,
556 boolean isMajor) throws IOException {
557 writer.appendMetadata(fd.maxSeqId, isMajor);
558 writer.close();
559 }
560 }