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.security.PrivilegedExceptionAction;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.concurrent.atomic.AtomicInteger;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Cell;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.KeyValue;
35 import org.apache.hadoop.hbase.KeyValueUtil;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.client.Scan;
38 import org.apache.hadoop.hbase.io.compress.Compression;
39 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
40 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
41 import org.apache.hadoop.hbase.regionserver.HStore;
42 import org.apache.hadoop.hbase.regionserver.InternalScanner;
43 import org.apache.hadoop.hbase.regionserver.ScanType;
44 import org.apache.hadoop.hbase.regionserver.Store;
45 import org.apache.hadoop.hbase.regionserver.StoreFile;
46 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
47 import org.apache.hadoop.hbase.regionserver.StoreScanner;
48 import org.apache.hadoop.hbase.security.User;
49 import org.apache.hadoop.hbase.util.Bytes;
50 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51 import org.apache.hadoop.util.StringUtils;
52
53
54
55
56
57 @InterfaceAudience.Private
58 public abstract class Compactor {
59 private static final Log LOG = LogFactory.getLog(Compactor.class);
60 protected CompactionProgress progress;
61 protected Configuration conf;
62 protected Store store;
63
64 private int compactionKVMax;
65 protected Compression.Algorithm compactionCompression;
66
67
68 Compactor(final Configuration conf, final Store store) {
69 this.conf = conf;
70 this.store = store;
71 this.compactionKVMax =
72 this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
73 this.compactionCompression = (this.store.getFamily() == null) ?
74 Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
75 }
76
77
78
79
80 public interface CellSink {
81 void append(KeyValue kv) throws IOException;
82 }
83
84 public CompactionProgress getProgress() {
85 return this.progress;
86 }
87
88
89 protected static class FileDetails {
90
91 public long maxKeyCount = 0;
92
93 public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
94
95 public long maxSeqId = 0;
96
97 public long maxMVCCReadpoint = 0;
98
99 public int maxTagsLength = 0;
100 }
101
102
103
104
105
106
107
108 protected FileDetails getFileDetails(
109 Collection<StoreFile> filesToCompact, boolean calculatePutTs) throws IOException {
110 FileDetails fd = new FileDetails();
111
112 for (StoreFile file : filesToCompact) {
113 long seqNum = file.getMaxSequenceId();
114 fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
115 StoreFile.Reader r = file.getReader();
116 if (r == null) {
117 LOG.warn("Null reader for " + file.getPath());
118 continue;
119 }
120
121
122
123 long keyCount = r.getEntries();
124 fd.maxKeyCount += keyCount;
125
126 Map<byte[], byte[]> fileInfo = r.loadFileInfo();
127 byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
128 if (tmp != null) {
129 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
130 }
131 tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
132 if (tmp != null) {
133 fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
134 }
135
136
137 long earliestPutTs = 0;
138 if (calculatePutTs) {
139 tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
140 if (tmp == null) {
141
142
143 fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
144 } else {
145 earliestPutTs = Bytes.toLong(tmp);
146 fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
147 }
148 }
149 if (LOG.isDebugEnabled()) {
150 LOG.debug("Compacting " + file +
151 ", keycount=" + keyCount +
152 ", bloomtype=" + r.getBloomFilterType().toString() +
153 ", size=" + StringUtils.humanReadableInt(r.length()) +
154 ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
155 ", seqNum=" + seqNum +
156 (calculatePutTs ? ", earliestPutTs=" + earliestPutTs: ""));
157 }
158 }
159 return fd;
160 }
161
162
163
164
165
166
167 protected List<StoreFileScanner> createFileScanners(
168 final Collection<StoreFile> filesToCompact,
169 long smallestReadPoint,
170 boolean useDropBehind) throws IOException {
171 return StoreFileScanner.getScannersForStoreFiles(filesToCompact,
172
173
174
175
176 smallestReadPoint);
177 }
178
179 protected long getSmallestReadPoint() {
180 return store.getSmallestReadPoint();
181 }
182
183
184
185
186
187
188
189
190
191 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
192 ScanType scanType, long earliestPutTs, List<StoreFileScanner> scanners) throws IOException {
193 return preCreateCoprocScanner(request, scanType, earliestPutTs, scanners, null);
194 }
195
196 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
197 final ScanType scanType, final long earliestPutTs, final List<StoreFileScanner> scanners,
198 User user) throws IOException {
199 if (store.getCoprocessorHost() == null) return null;
200 if (user == null) {
201 return store.getCoprocessorHost().preCompactScannerOpen(store, scanners, scanType,
202 earliestPutTs, request);
203 } else {
204 try {
205 return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
206 @Override
207 public InternalScanner run() throws Exception {
208 return store.getCoprocessorHost().preCompactScannerOpen(store, scanners,
209 scanType, earliestPutTs, request);
210 }
211 });
212 } catch (InterruptedException ie) {
213 InterruptedIOException iioe = new InterruptedIOException();
214 iioe.initCause(ie);
215 throw iioe;
216 }
217 }
218 }
219
220
221
222
223
224
225
226
227 protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
228 final ScanType scanType, final InternalScanner scanner, User user) throws IOException {
229 if (store.getCoprocessorHost() == null) return scanner;
230 if (user == null) {
231 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
232 } else {
233 try {
234 return user.getUGI().doAs(new PrivilegedExceptionAction<InternalScanner>() {
235 @Override
236 public InternalScanner run() throws Exception {
237 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
238 }
239 });
240 } catch (InterruptedException ie) {
241 InterruptedIOException iioe = new InterruptedIOException();
242 iioe.initCause(ie);
243 throw iioe;
244 }
245 }
246 }
247
248
249
250
251
252 private static final AtomicInteger NAME_COUNTER = new AtomicInteger(0);
253
254 private String generateCompactionName() {
255 int counter;
256 for (;;) {
257 counter = NAME_COUNTER.get();
258 int next = counter == Integer.MAX_VALUE ? 0 : counter + 1;
259 if (NAME_COUNTER.compareAndSet(counter, next)) {
260 break;
261 }
262 }
263 return store.getRegionInfo().getRegionNameAsString() + "#"
264 + store.getFamily().getNameAsString() + "#" + counter;
265 }
266
267
268
269
270
271
272
273 protected boolean performCompaction(InternalScanner scanner, CellSink writer,
274 long smallestReadPoint, CompactionThroughputController throughputController)
275 throws IOException {
276 long bytesWritten = 0;
277 long bytesWrittenProgress = 0;
278
279
280 List<Cell> kvs = new ArrayList<Cell>();
281 long closeCheckInterval = HStore.getCloseCheckInterval();
282 long lastMillis = 0;
283 if (LOG.isDebugEnabled()) {
284 lastMillis = EnvironmentEdgeManager.currentTimeMillis();
285 }
286 String compactionName = generateCompactionName();
287 long now = 0;
288 boolean hasMore;
289 throughputController.start(compactionName);
290 try {
291 do {
292 hasMore = scanner.next(kvs, compactionKVMax);
293 if (LOG.isDebugEnabled()) {
294 now = EnvironmentEdgeManager.currentTimeMillis();
295 }
296
297 for (Cell c : kvs) {
298 KeyValue kv = KeyValueUtil.ensureKeyValue(c);
299 if (kv.getMvccVersion() <= smallestReadPoint) {
300 kv.setMvccVersion(0);
301 }
302 writer.append(kv);
303 int len = kv.getLength();
304 ++progress.currentCompactedKVs;
305 progress.totalCompactedSize += len;
306 if (LOG.isDebugEnabled()) {
307 bytesWrittenProgress += len;
308 }
309 throughputController.control(compactionName, len);
310
311
312 if (closeCheckInterval > 0) {
313 bytesWritten += len;
314 if (bytesWritten > closeCheckInterval) {
315 bytesWritten = 0;
316 if (!store.areWritesEnabled()) {
317 progress.cancel();
318 return false;
319 }
320 }
321 }
322 }
323
324
325 if (LOG.isDebugEnabled()) {
326 if ((now - lastMillis) >= 60 * 1000) {
327 LOG.debug("Compaction progress: "
328 + compactionName
329 + " "
330 + progress
331 + String.format(", rate=%.2f kB/sec", (bytesWrittenProgress / 1024.0)
332 / ((now - lastMillis) / 1000.0)) + ", throughputController is "
333 + throughputController);
334 lastMillis = now;
335 bytesWrittenProgress = 0;
336 }
337 }
338 kvs.clear();
339 } while (hasMore);
340 } catch (InterruptedException e) {
341 throw new InterruptedIOException("Interrupted while control throughput of compacting "
342 + compactionName);
343 } finally {
344 throughputController.finish(compactionName);
345 progress.complete();
346 }
347 return true;
348 }
349
350
351
352
353
354
355
356
357
358 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
359 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
360 Scan scan = new Scan();
361 scan.setMaxVersions(store.getFamily().getMaxVersions());
362 return new StoreScanner(store, store.getScanInfo(), scan, scanners,
363 scanType, smallestReadPoint, earliestPutTs);
364 }
365
366
367
368
369
370
371
372
373
374
375 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
376 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
377 byte[] dropDeletesToRow) throws IOException {
378 Scan scan = new Scan();
379 scan.setMaxVersions(store.getFamily().getMaxVersions());
380 return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
381 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
382 }
383 }