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