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