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.isAllFiles());
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 boolean cleanSeqId = false;
77 IOException e = null;
78 try {
79 InternalScanner scanner = null;
80 try {
81
82 ScanType scanType =
83 request.isAllFiles() ? ScanType.COMPACT_DROP_DELETES : ScanType.COMPACT_RETAIN_DELETES;
84 scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
85 if (scanner == null) {
86 scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs);
87 }
88 scanner = postCreateCoprocScanner(request, scanType, scanner);
89 if (scanner == null) {
90
91 return newFiles;
92 }
93
94
95 if(fd.minSeqIdToKeep > 0) {
96 smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
97 cleanSeqId = true;
98 }
99
100 writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, true,
101 true, fd.maxTagsLength > 0);
102 boolean finished = performCompaction(scanner, writer, smallestReadPoint, cleanSeqId);
103 if (!finished) {
104 writer.close();
105 store.getFileSystem().delete(writer.getPath(), false);
106 writer = null;
107 throw new InterruptedIOException( "Aborting compaction of store " + store +
108 " in region " + store.getRegionInfo().getRegionNameAsString() +
109 " because it was interrupted.");
110 }
111 } finally {
112 if (scanner != null) {
113 scanner.close();
114 }
115 }
116 } catch (IOException ioe) {
117 e = ioe;
118
119 throw ioe;
120 }
121 finally {
122 try {
123 if (writer != null) {
124 if (e != null) {
125 writer.close();
126 } else {
127 writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
128 writer.close();
129 newFiles.add(writer.getPath());
130 }
131 }
132 } finally {
133 for (StoreFile f : readersToClose) {
134 try {
135 f.closeReader(true);
136 } catch (IOException ioe) {
137 LOG.warn("Exception closing " + f, ioe);
138 }
139 }
140 }
141 }
142 return newFiles;
143 }
144
145
146
147
148
149
150
151
152
153
154
155 public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
156 throws IOException {
157 CompactionRequest cr = new CompactionRequest(filesToCompact);
158 cr.setIsMajor(isMajor, isMajor);
159 return this.compact(cr);
160 }
161 }