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
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.Path;
32 import org.apache.hadoop.hbase.regionserver.InternalScanner;
33 import org.apache.hadoop.hbase.regionserver.ScanType;
34 import org.apache.hadoop.hbase.regionserver.Store;
35 import org.apache.hadoop.hbase.regionserver.StoreFile;
36 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
37
38
39
40
41 @InterfaceAudience.Private
42 public class DefaultCompactor extends Compactor {
43 private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
44
45 public DefaultCompactor(final Configuration conf, final Store store) {
46 super(conf, store);
47 }
48
49
50
51
52 public List<Path> compact(final CompactionRequest request) throws IOException {
53 FileDetails fd = getFileDetails(request.getFiles(), request.isMajor());
54 this.progress = new CompactionProgress(fd.maxKeyCount);
55
56
57 long smallestReadPoint = getSmallestReadPoint();
58
59 List<StoreFileScanner> scanners;
60 Collection<StoreFile> readersToClose;
61 if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", false)) {
62
63
64 readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
65 for (StoreFile f : request.getFiles()) {
66 readersToClose.add(new StoreFile(f));
67 }
68 scanners = createFileScanners(readersToClose, smallestReadPoint);
69 } else {
70 readersToClose = Collections.emptyList();
71 scanners = createFileScanners(request.getFiles(), smallestReadPoint);
72 }
73
74 StoreFile.Writer writer = null;
75 List<Path> newFiles = new ArrayList<Path>();
76 IOException e = null;
77 try {
78 InternalScanner scanner = null;
79 try {
80
81 ScanType scanType =
82 request.isMajor() ? ScanType.COMPACT_DROP_DELETES : ScanType.COMPACT_RETAIN_DELETES;
83 scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
84 if (scanner == null) {
85 scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs);
86 }
87 scanner = postCreateCoprocScanner(request, scanType, scanner);
88 if (scanner == null) {
89
90 return newFiles;
91 }
92
93
94 writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, true,
95 fd.maxMVCCReadpoint >= smallestReadPoint, fd.maxTagsLength > 0);
96 boolean finished = performCompaction(scanner, writer, smallestReadPoint);
97 if (!finished) {
98 writer.close();
99 store.getFileSystem().delete(writer.getPath(), false);
100 writer = null;
101 throw new InterruptedIOException( "Aborting compaction of store " + store +
102 " in region " + store.getRegionInfo().getRegionNameAsString() +
103 " because it was interrupted.");
104 }
105 } finally {
106 if (scanner != null) {
107 scanner.close();
108 }
109 }
110 } catch (IOException ioe) {
111 e = ioe;
112
113 throw ioe;
114 } finally {
115 try {
116 if (writer != null) {
117 if (e != null) {
118 writer.close();
119 } else {
120 writer.appendMetadata(fd.maxSeqId, request.isMajor());
121 writer.close();
122 newFiles.add(writer.getPath());
123 }
124 }
125 } finally {
126 for (StoreFile f : readersToClose) {
127 try {
128 f.closeReader(true);
129 } catch (IOException ioe) {
130 LOG.warn("Exception closing " + f, ioe);
131 }
132 }
133 }
134 }
135 return newFiles;
136 }
137
138
139
140
141
142
143
144
145
146
147
148 public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
149 throws IOException {
150 CompactionRequest cr = new CompactionRequest(filesToCompact);
151 cr.setIsMajor(isMajor);
152 return this.compact(cr);
153 }
154 }