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