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
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.fs.FileSystem;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
29 import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory;
30 import org.apache.hadoop.hbase.regionserver.InternalScanner;
31 import org.apache.hadoop.hbase.regionserver.Store;
32 import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
33 import org.apache.hadoop.hbase.regionserver.StoreScanner;
34
35
36
37
38
39 @InterfaceAudience.Private
40 public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWriter>
41 extends Compactor<T> {
42
43 private static final Log LOG = LogFactory.getLog(AbstractMultiOutputCompactor.class);
44
45 public AbstractMultiOutputCompactor(Configuration conf, Store store) {
46 super(conf, store);
47 }
48
49 protected void initMultiWriter(AbstractMultiFileWriter writer, InternalScanner scanner,
50 final FileDetails fd, final boolean shouldDropBehind) {
51 WriterFactory writerFactory = new WriterFactory() {
52 @Override
53 public Writer createWriter() throws IOException {
54 return createTmpWriter(fd, shouldDropBehind);
55 }
56 };
57
58
59 StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner) scanner : null;
60 writer.init(storeScanner, writerFactory);
61 }
62
63 @Override
64 protected void abortWriter(T writer) throws IOException {
65 FileSystem fs = store.getFileSystem();
66 for (Path leftoverFile : writer.abortWriters()) {
67 try {
68 fs.delete(leftoverFile, false);
69 } catch (IOException e) {
70 LOG.warn(
71 "Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.",
72 e);
73 }
74 }
75 }
76 }