1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
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.List;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.conf.Configured;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.KeyValue;
33 import org.apache.hadoop.hbase.client.Scan;
34 import org.apache.hadoop.hbase.io.hfile.Compression;
35 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
36 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.util.StringUtils;
39
40
41
42
43
44 @InterfaceAudience.Private
45 class Compactor extends Configured {
46 private static final Log LOG = LogFactory.getLog(Compactor.class);
47 private CompactionProgress progress;
48
49 Compactor(final Configuration c) {
50 super(c);
51 }
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67 public StoreFile.Writer compactForTesting(final Store store, Configuration conf,
68 final Collection<StoreFile> filesToCompact,
69 boolean isMajor, long maxId) throws IOException {
70 return compact(CompactionRequest.getRequestForTesting(store, conf, filesToCompact, isMajor),
71 maxId);
72 }
73
74
75
76
77
78
79
80
81
82 StoreFile.Writer compact(CompactionRequest request, long maxId) throws IOException {
83
84
85 int maxKeyCount = 0;
86 long earliestPutTs = HConstants.LATEST_TIMESTAMP;
87
88
89 final Store store = request.getStore();
90 final boolean majorCompaction = request.isMajor();
91 final List<StoreFile> filesToCompact = request.getFiles();
92
93 for (StoreFile file : filesToCompact) {
94 StoreFile.Reader r = file.getReader();
95 if (r == null) {
96 LOG.warn("Null reader for " + file.getPath());
97 continue;
98 }
99
100
101 long keyCount = (r.getBloomFilterType() == store.getFamily()
102 .getBloomFilterType()) ?
103 r.getFilterEntries() : r.getEntries();
104 maxKeyCount += keyCount;
105
106
107
108 if (majorCompaction) {
109 byte[] tmp = r.loadFileInfo().get(StoreFile.EARLIEST_PUT_TS);
110 if (tmp == null) {
111
112
113 earliestPutTs = HConstants.OLDEST_TIMESTAMP;
114 } else {
115 earliestPutTs = Math.min(earliestPutTs, Bytes.toLong(tmp));
116 }
117 }
118 if (LOG.isDebugEnabled()) {
119 LOG.debug("Compacting " + file +
120 ", keycount=" + keyCount +
121 ", bloomtype=" + r.getBloomFilterType().toString() +
122 ", size=" + StringUtils.humanReadableInt(r.length()) +
123 ", encoding=" + r.getHFileReader().getEncodingOnDisk() +
124 (majorCompaction? ", earliestPutTs=" + earliestPutTs: ""));
125 }
126 }
127
128
129 this.progress = new CompactionProgress(maxKeyCount);
130
131 int compactionKVMax = getConf().getInt("hbase.hstore.compaction.kv.max", 10);
132 Compression.Algorithm compression = store.getFamily().getCompression();
133
134
135 Compression.Algorithm compactionCompression =
136 (store.getFamily().getCompactionCompression() != Compression.Algorithm.NONE) ?
137 store.getFamily().getCompactionCompression(): compression;
138
139
140 List<StoreFileScanner> scanners = StoreFileScanner
141 .getScannersForStoreFiles(filesToCompact, false, false, true);
142
143
144
145 StoreFile.Writer writer = null;
146
147 long smallestReadPoint = store.getHRegion().getSmallestReadPoint();
148 MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
149 try {
150 InternalScanner scanner = null;
151 try {
152 if (store.getHRegion().getCoprocessorHost() != null) {
153 scanner = store.getHRegion()
154 .getCoprocessorHost()
155 .preCompactScannerOpen(store, scanners,
156 majorCompaction ? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT, earliestPutTs,
157 request);
158 }
159 if (scanner == null) {
160 Scan scan = new Scan();
161 scan.setMaxVersions(store.getFamily().getMaxVersions());
162
163 scanner = new StoreScanner(store, store.getScanInfo(), scan, scanners,
164 majorCompaction? ScanType.MAJOR_COMPACT : ScanType.MINOR_COMPACT,
165 smallestReadPoint, earliestPutTs);
166 }
167 if (store.getHRegion().getCoprocessorHost() != null) {
168 InternalScanner cpScanner =
169 store.getHRegion().getCoprocessorHost().preCompact(store, scanner, request);
170
171 if (cpScanner == null) {
172 return null;
173 }
174 scanner = cpScanner;
175 }
176
177 int bytesWritten = 0;
178
179
180 List<KeyValue> kvs = new ArrayList<KeyValue>();
181
182 boolean hasMore;
183 do {
184 hasMore = scanner.next(kvs, compactionKVMax);
185 if (writer == null && !kvs.isEmpty()) {
186 writer = store.createWriterInTmp(maxKeyCount, compactionCompression, true);
187 }
188 if (writer != null) {
189
190 for (KeyValue kv : kvs) {
191 if (kv.getMemstoreTS() <= smallestReadPoint) {
192 kv.setMemstoreTS(0);
193 }
194 writer.append(kv);
195
196 ++progress.currentCompactedKVs;
197
198
199 if (Store.closeCheckInterval > 0) {
200 bytesWritten += kv.getLength();
201 if (bytesWritten > Store.closeCheckInterval) {
202 bytesWritten = 0;
203 isInterrupted(store, writer);
204 }
205 }
206 }
207 }
208 kvs.clear();
209 } while (hasMore);
210 } finally {
211 if (scanner != null) {
212 scanner.close();
213 }
214 }
215 } finally {
216 if (writer != null) {
217 writer.appendMetadata(maxId, majorCompaction);
218 writer.close();
219 }
220 }
221 return writer;
222 }
223
224 void isInterrupted(final Store store, final StoreFile.Writer writer)
225 throws IOException {
226 if (store.getHRegion().areWritesEnabled()) return;
227
228 writer.close();
229 store.getFileSystem().delete(writer.getPath(), false);
230 throw new InterruptedIOException( "Aborting compaction of store " + store +
231 " in region " + store.getHRegion() + " because user requested stop.");
232 }
233
234 CompactionProgress getProgress() {
235 return this.progress;
236 }
237 }