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