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