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