1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile;
21
22 import java.io.ByteArrayOutputStream;
23 import java.io.DataOutputStream;
24 import java.io.IOException;
25 import java.io.OutputStream;
26 import java.nio.ByteBuffer;
27 import java.util.ArrayList;
28 import java.util.List;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.FSDataOutputStream;
34 import org.apache.hadoop.fs.FileSystem;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
38 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
39 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
40 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
41 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
42 import org.apache.hadoop.hbase.regionserver.MemStore;
43 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
44 import org.apache.hadoop.hbase.util.ChecksumType;
45 import org.apache.hadoop.hbase.util.BloomFilterWriter;
46 import org.apache.hadoop.hbase.util.Bytes;
47 import org.apache.hadoop.io.Writable;
48 import org.apache.hadoop.io.compress.Compressor;
49
50
51
52
53 public class HFileWriterV1 extends AbstractHFileWriter {
54
55
56 static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
57
58
59 public static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
60
61 private static final Log LOG = LogFactory.getLog(HFileWriterV1.class);
62
63
64 private DataOutputStream out;
65
66
67 private long blockBegin;
68
69
70 private ArrayList<byte[]> blockKeys = new ArrayList<byte[]>();
71
72
73 private ArrayList<Long> blockOffsets = new ArrayList<Long>();
74
75
76 private ArrayList<Integer> blockDataSizes = new ArrayList<Integer>();
77
78 private Compressor compressor;
79
80
81 private ByteArrayOutputStream baos;
82 private DataOutputStream baosDos;
83 private int blockNumber = 0;
84
85 static class WriterFactoryV1 extends HFile.WriterFactory {
86 WriterFactoryV1(Configuration conf, CacheConfig cacheConf) {
87 super(conf, cacheConf);
88 }
89
90 @Override
91 public Writer createWriter(FileSystem fs, Path path,
92 FSDataOutputStream ostream, int blockSize,
93 Algorithm compressAlgo, HFileDataBlockEncoder dataBlockEncoder,
94 KeyComparator comparator, final ChecksumType checksumType,
95 final int bytesPerChecksum) throws IOException {
96
97 return new HFileWriterV1(conf, cacheConf, fs, path, ostream, blockSize,
98 compressAlgo, dataBlockEncoder, comparator);
99 }
100 }
101
102
103 public HFileWriterV1(Configuration conf, CacheConfig cacheConf,
104 FileSystem fs, Path path, FSDataOutputStream ostream,
105 int blockSize, Compression.Algorithm compress,
106 HFileDataBlockEncoder blockEncoder,
107 final KeyComparator comparator) throws IOException {
108 super(cacheConf, ostream == null ? createOutputStream(conf, fs, path) : ostream, path,
109 blockSize, compress, blockEncoder, comparator);
110 SchemaMetrics.configureGlobally(conf);
111 }
112
113
114
115
116
117
118 private void checkBlockBoundary() throws IOException {
119 if (this.out != null && this.out.size() < blockSize)
120 return;
121 finishBlock();
122 newBlock();
123 }
124
125
126
127
128
129
130 private void finishBlock() throws IOException {
131 if (this.out == null)
132 return;
133 long startTimeNs = System.nanoTime();
134
135 int size = releaseCompressingStream(this.out);
136 this.out = null;
137 blockKeys.add(firstKeyInBlock);
138 blockOffsets.add(Long.valueOf(blockBegin));
139 blockDataSizes.add(Integer.valueOf(size));
140 this.totalUncompressedBytes += size;
141
142 HFile.offerWriteLatency(System.nanoTime() - startTimeNs);
143
144 if (cacheConf.shouldCacheDataOnWrite()) {
145 baosDos.flush();
146
147 byte[] bytes = baos.toByteArray();
148 HFileBlock block = new HFileBlock(BlockType.DATA,
149 (int) (outputStream.getPos() - blockBegin), bytes.length, -1,
150 ByteBuffer.wrap(bytes, 0, bytes.length), HFileBlock.FILL_HEADER,
151 blockBegin, MemStore.NO_PERSISTENT_TS,
152 HFileBlock.MINOR_VERSION_NO_CHECKSUM,
153 0,
154 ChecksumType.NULL.getCode(),
155 (int) (outputStream.getPos() - blockBegin) +
156 HFileBlock.HEADER_SIZE_NO_CHECKSUM);
157
158 block = blockEncoder.diskToCacheFormat(block, false);
159 passSchemaMetricsTo(block);
160 cacheConf.getBlockCache().cacheBlock(
161 new BlockCacheKey(name, blockBegin, DataBlockEncoding.NONE,
162 block.getBlockType()), block);
163 baosDos.close();
164 }
165 blockNumber++;
166 }
167
168
169
170
171
172
173 private void newBlock() throws IOException {
174
175 blockBegin = outputStream.getPos();
176 this.out = getCompressingStream();
177 BlockType.DATA.write(out);
178 firstKeyInBlock = null;
179 if (cacheConf.shouldCacheDataOnWrite()) {
180 this.baos = new ByteArrayOutputStream();
181 this.baosDos = new DataOutputStream(baos);
182 baosDos.write(HFileBlock.DUMMY_HEADER_NO_CHECKSUM);
183 }
184 }
185
186
187
188
189
190
191
192
193
194
195
196
197 private DataOutputStream getCompressingStream() throws IOException {
198 this.compressor = compressAlgo.getCompressor();
199
200
201
202
203
204
205
206
207 OutputStream os = this.compressAlgo.createCompressionStream(
208 this.outputStream, this.compressor, 0);
209 return new DataOutputStream(os);
210 }
211
212
213
214
215
216
217
218
219
220
221
222
223
224 private int releaseCompressingStream(final DataOutputStream dos)
225 throws IOException {
226 dos.flush();
227 this.compressAlgo.returnCompressor(this.compressor);
228 this.compressor = null;
229 return dos.size();
230 }
231
232
233
234
235
236
237
238
239
240
241
242
243 public void appendMetaBlock(String metaBlockName, Writable content) {
244 byte[] key = Bytes.toBytes(metaBlockName);
245 int i;
246 for (i = 0; i < metaNames.size(); ++i) {
247
248 byte[] cur = metaNames.get(i);
249 if (Bytes.BYTES_RAWCOMPARATOR.compare(cur, 0, cur.length, key, 0,
250 key.length) > 0) {
251 break;
252 }
253 }
254 metaNames.add(i, key);
255 metaData.add(i, content);
256 }
257
258
259
260
261
262
263
264
265
266 public void append(final KeyValue kv) throws IOException {
267 append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
268 kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
269 }
270
271
272
273
274
275
276
277
278
279
280
281 public void append(final byte[] key, final byte[] value) throws IOException {
282 append(key, 0, key.length, value, 0, value.length);
283 }
284
285
286
287
288
289
290
291
292
293
294
295
296
297 private void append(final byte[] key, final int koffset, final int klength,
298 final byte[] value, final int voffset, final int vlength)
299 throws IOException {
300 boolean dupKey = checkKey(key, koffset, klength);
301 checkValue(value, voffset, vlength);
302 if (!dupKey) {
303 checkBlockBoundary();
304 }
305
306 this.out.writeInt(klength);
307 totalKeyLength += klength;
308 this.out.writeInt(vlength);
309 totalValueLength += vlength;
310 this.out.write(key, koffset, klength);
311 this.out.write(value, voffset, vlength);
312
313 if (this.firstKeyInBlock == null) {
314
315 this.firstKeyInBlock = new byte[klength];
316 System.arraycopy(key, koffset, this.firstKeyInBlock, 0, klength);
317 }
318 this.lastKeyBuffer = key;
319 this.lastKeyOffset = koffset;
320 this.lastKeyLength = klength;
321 this.entryCount++;
322
323 if (cacheConf.shouldCacheDataOnWrite()) {
324 this.baosDos.writeInt(klength);
325 this.baosDos.writeInt(vlength);
326 this.baosDos.write(key, koffset, klength);
327 this.baosDos.write(value, voffset, vlength);
328 }
329 }
330
331 public void close() throws IOException {
332 if (this.outputStream == null) {
333 return;
334 }
335
336 blockEncoder.saveMetadata(this);
337
338
339
340 finishBlock();
341
342 FixedFileTrailer trailer = new FixedFileTrailer(1,
343 HFileBlock.MINOR_VERSION_NO_CHECKSUM);
344
345
346 ArrayList<Long> metaOffsets = null;
347 ArrayList<Integer> metaDataSizes = null;
348 if (metaNames.size() > 0) {
349 metaOffsets = new ArrayList<Long>(metaNames.size());
350 metaDataSizes = new ArrayList<Integer>(metaNames.size());
351 for (int i = 0; i < metaNames.size(); ++i) {
352
353 long curPos = outputStream.getPos();
354 metaOffsets.add(curPos);
355
356 DataOutputStream dos = getCompressingStream();
357 BlockType.META.write(dos);
358 metaData.get(i).write(dos);
359 int size = releaseCompressingStream(dos);
360
361 metaDataSizes.add(size);
362 }
363 }
364
365 writeFileInfo(trailer, outputStream);
366
367
368 trailer.setLoadOnOpenOffset(writeBlockIndex(this.outputStream,
369 this.blockKeys, this.blockOffsets, this.blockDataSizes));
370 LOG.info("Wrote a version 1 block index with " + this.blockKeys.size()
371 + " keys");
372
373 if (metaNames.size() > 0) {
374
375 writeBlockIndex(this.outputStream, metaNames, metaOffsets, metaDataSizes);
376 }
377
378
379 trailer.setDataIndexCount(blockKeys.size());
380
381 finishClose(trailer);
382 }
383
384 @Override
385 protected void finishFileInfo() throws IOException {
386 super.finishFileInfo();
387
388
389 fileInfo.append(FileInfo.COMPARATOR,
390 Bytes.toBytes(comparator.getClass().getName()), false);
391 }
392
393 @Override
394 public void addInlineBlockWriter(InlineBlockWriter bloomWriter) {
395
396 throw new UnsupportedOperationException();
397 }
398
399
400
401
402
403 @Override
404 public void addGeneralBloomFilter(BloomFilterWriter bfw) {
405 appendMetaBlock(BLOOM_FILTER_META_KEY,
406 bfw.getMetaWriter());
407 Writable dataWriter = bfw.getDataWriter();
408 if (dataWriter != null) {
409 appendMetaBlock(BLOOM_FILTER_DATA_KEY, dataWriter);
410 }
411 }
412
413 @Override
414 public void addDeleteFamilyBloomFilter(BloomFilterWriter bfw)
415 throws IOException {
416 throw new IOException("Delete Bloom filter is not supported in HFile V1");
417 }
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433 private static long writeBlockIndex(final FSDataOutputStream out,
434 final List<byte[]> keys, final List<Long> offsets,
435 final List<Integer> uncompressedSizes) throws IOException {
436 long pos = out.getPos();
437
438 if (keys.size() > 0) {
439 BlockType.INDEX_V1.write(out);
440
441 for (int i = 0; i < keys.size(); ++i) {
442 out.writeLong(offsets.get(i).longValue());
443 out.writeInt(uncompressedSizes.get(i).intValue());
444 byte[] key = keys.get(i);
445 Bytes.writeByteArray(out, key);
446 }
447 }
448 return pos;
449 }
450
451 }