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.util.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24 import java.util.Map;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.Cell;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.KeyValue;
33 import org.apache.hadoop.hbase.KeyValueUtil;
34 import org.apache.hadoop.hbase.client.Scan;
35 import org.apache.hadoop.hbase.io.compress.Compression;
36 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
37 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
38 import org.apache.hadoop.hbase.regionserver.HStore;
39 import org.apache.hadoop.hbase.regionserver.InternalScanner;
40 import org.apache.hadoop.hbase.regionserver.ScanType;
41 import org.apache.hadoop.hbase.regionserver.Store;
42 import org.apache.hadoop.hbase.regionserver.StoreFile;
43 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
44 import org.apache.hadoop.hbase.regionserver.StoreScanner;
45 import org.apache.hadoop.hbase.util.Bytes;
46 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
47 import org.apache.hadoop.util.StringUtils;
48
49
50
51
52
53 @InterfaceAudience.Private
54 public abstract class Compactor {
55 private static final Log LOG = LogFactory.getLog(Compactor.class);
56 protected CompactionProgress progress;
57 protected Configuration conf;
58 protected Store store;
59
60 private int compactionKVMax;
61 protected Compression.Algorithm compactionCompression;
62
63
64 Compactor(final Configuration conf, final Store store) {
65 this.conf = conf;
66 this.store = store;
67 this.compactionKVMax =
68 this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
69 this.compactionCompression = (this.store.getFamily() == null) ?
70 Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
71 }
72
73
74
75
76 public interface CellSink {
77 void append(KeyValue kv) throws IOException;
78 }
79
80 public CompactionProgress getProgress() {
81 return this.progress;
82 }
83
84
85 protected static class FileDetails {
86
87 public long maxKeyCount = 0;
88
89 public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
90
91 public long maxSeqId = 0;
92
93 public long maxMVCCReadpoint = 0;
94
95 public int maxTagsLength = 0;
96 }
97
98
99
100
101
102
103
104 protected FileDetails getFileDetails(
105 Collection<StoreFile> filesToCompact, boolean calculatePutTs) throws IOException {
106 FileDetails fd = new FileDetails();
107
108 for (StoreFile file : filesToCompact) {
109 long seqNum = file.getMaxSequenceId();
110 fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
111 StoreFile.Reader r = file.getReader();
112 if (r == null) {
113 LOG.warn("Null reader for " + file.getPath());
114 continue;
115 }
116
117
118 long keyCount = (r.getBloomFilterType() == store.getFamily().getBloomFilterType())
119 ? r.getFilterEntries() : r.getEntries();
120 fd.maxKeyCount += keyCount;
121
122 Map<byte[], byte[]> fileInfo = r.loadFileInfo();
123 byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
124 if (tmp != null) {
125 fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
126 }
127 tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
128 if (tmp != null) {
129 fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
130 }
131
132
133 long earliestPutTs = 0;
134 if (calculatePutTs) {
135 tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
136 if (tmp == null) {
137
138
139 fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
140 } else {
141 earliestPutTs = Bytes.toLong(tmp);
142 fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
143 }
144 }
145 if (LOG.isDebugEnabled()) {
146 LOG.debug("Compacting " + file +
147 ", keycount=" + keyCount +
148 ", bloomtype=" + r.getBloomFilterType().toString() +
149 ", size=" + StringUtils.humanReadableInt(r.length()) +
150 ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
151 ", seqNum=" + seqNum +
152 (calculatePutTs ? ", earliestPutTs=" + earliestPutTs: ""));
153 }
154 }
155 return fd;
156 }
157
158
159
160
161
162
163 protected List<StoreFileScanner> createFileScanners(
164 final Collection<StoreFile> filesToCompact, long smallestReadPoint) throws IOException {
165 return StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true,
166 smallestReadPoint);
167 }
168
169 protected long getSmallestReadPoint() {
170 return store.getSmallestReadPoint();
171 }
172
173
174
175
176
177
178
179
180
181 protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
182 ScanType scanType, long earliestPutTs, List<StoreFileScanner> scanners) throws IOException {
183 if (store.getCoprocessorHost() == null) return null;
184 return store.getCoprocessorHost()
185 .preCompactScannerOpen(store, scanners, scanType, earliestPutTs, request);
186 }
187
188
189
190
191
192
193
194
195 protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
196 ScanType scanType, InternalScanner scanner) throws IOException {
197 if (store.getCoprocessorHost() == null) return scanner;
198 return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
199 }
200
201
202
203
204
205
206
207
208 protected boolean performCompaction(InternalScanner scanner,
209 CellSink writer, long smallestReadPoint) throws IOException {
210 long bytesWritten = 0;
211 long bytesWrittenProgress = 0;
212
213
214 List<Cell> kvs = new ArrayList<Cell>();
215 long closeCheckInterval = HStore.getCloseCheckInterval();
216 long lastMillis = 0;
217 if (LOG.isDebugEnabled()) {
218 lastMillis = EnvironmentEdgeManager.currentTimeMillis();
219 }
220 long now = 0;
221 boolean hasMore;
222 do {
223 hasMore = scanner.next(kvs, compactionKVMax);
224 if (LOG.isDebugEnabled()) {
225 now = EnvironmentEdgeManager.currentTimeMillis();
226 }
227
228 for (Cell c : kvs) {
229 KeyValue kv = KeyValueUtil.ensureKeyValue(c);
230 if (kv.getMvccVersion() <= smallestReadPoint) {
231 kv.setMvccVersion(0);
232 }
233 writer.append(kv);
234 int len = kv.getLength();
235 ++progress.currentCompactedKVs;
236 progress.totalCompactedSize += len;
237 if (LOG.isDebugEnabled()) {
238 bytesWrittenProgress += len;
239 }
240
241
242 if (closeCheckInterval > 0) {
243 bytesWritten += len;
244 if (bytesWritten > closeCheckInterval) {
245 bytesWritten = 0;
246 if (!store.areWritesEnabled()) {
247 progress.cancel();
248 return false;
249 }
250 }
251 }
252 }
253
254
255 if (LOG.isDebugEnabled()) {
256 if ((now - lastMillis) >= 60 * 1000) {
257 LOG.debug("Compaction progress: " + progress + String.format(", rate=%.2f kB/sec",
258 (bytesWrittenProgress / 1024.0) / ((now - lastMillis) / 1000.0)));
259 lastMillis = now;
260 bytesWrittenProgress = 0;
261 }
262 }
263 kvs.clear();
264 } while (hasMore);
265 progress.complete();
266 return true;
267 }
268
269
270
271
272
273
274
275
276
277 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
278 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
279 Scan scan = new Scan();
280 scan.setMaxVersions(store.getFamily().getMaxVersions());
281 return new StoreScanner(store, store.getScanInfo(), scan, scanners,
282 scanType, smallestReadPoint, earliestPutTs);
283 }
284
285
286
287
288
289
290
291
292
293
294 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
295 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
296 byte[] dropDeletesToRow) throws IOException {
297 Scan scan = new Scan();
298 scan.setMaxVersions(store.getFamily().getMaxVersions());
299 return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
300 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
301 }
302 }