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 static org.junit.Assert.*;
23
24 import java.io.ByteArrayOutputStream;
25 import java.io.DataOutputStream;
26 import java.io.IOException;
27 import java.io.OutputStream;
28 import java.nio.ByteBuffer;
29 import java.util.ArrayList;
30 import java.util.Collection;
31 import java.util.List;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.fs.FSDataInputStream;
36 import org.apache.hadoop.fs.FSDataOutputStream;
37 import org.apache.hadoop.fs.Path;
38 import org.apache.hadoop.hbase.HBaseTestingUtility;
39 import org.apache.hadoop.hbase.MediumTests;
40 import org.apache.hadoop.hbase.fs.HFileSystem;
41 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.io.compress.CompressionOutputStream;
44 import org.apache.hadoop.io.compress.Compressor;
45 import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
46 import org.apache.hadoop.hbase.util.ChecksumType;
47 import org.apache.hadoop.hbase.util.Pair;
48 import com.google.common.base.Preconditions;
49
50 import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*;
51 import org.junit.Before;
52 import org.junit.Test;
53 import org.junit.experimental.categories.Category;
54 import org.junit.runner.RunWith;
55 import org.junit.runners.Parameterized;
56 import org.junit.runners.Parameterized.Parameters;
57
58
59
60
61
62 @Category(MediumTests.class)
63 @RunWith(Parameterized.class)
64 public class TestHFileBlockCompatibility {
65
66 private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true };
67
68 private static final Log LOG = LogFactory.getLog(TestHFileBlockCompatibility.class);
69
70 private static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
71 NONE, GZ };
72
73
74 private static int MINOR_VERSION = 0;
75
76 private static final HBaseTestingUtility TEST_UTIL =
77 new HBaseTestingUtility();
78 private HFileSystem fs;
79 private int uncompressedSizeV1;
80
81 private final boolean includesMemstoreTS;
82
83 public TestHFileBlockCompatibility(boolean includesMemstoreTS) {
84 this.includesMemstoreTS = includesMemstoreTS;
85 }
86
87 @Parameters
88 public static Collection<Object[]> parameters() {
89 return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
90 }
91
92 @Before
93 public void setUp() throws IOException {
94 fs = (HFileSystem)HFileSystem.get(TEST_UTIL.getConfiguration());
95 }
96
97 public byte[] createTestV1Block(Compression.Algorithm algo)
98 throws IOException {
99 Compressor compressor = algo.getCompressor();
100 ByteArrayOutputStream baos = new ByteArrayOutputStream();
101 OutputStream os = algo.createCompressionStream(baos, compressor, 0);
102 DataOutputStream dos = new DataOutputStream(os);
103 BlockType.META.write(dos);
104 TestHFileBlock.writeTestBlockContents(dos);
105 uncompressedSizeV1 = dos.size();
106 dos.flush();
107 algo.returnCompressor(compressor);
108 return baos.toByteArray();
109 }
110
111 private Writer createTestV2Block(Compression.Algorithm algo)
112 throws IOException {
113 final BlockType blockType = BlockType.DATA;
114 Writer hbw = new Writer(algo, null,
115 includesMemstoreTS);
116 DataOutputStream dos = hbw.startWriting(blockType);
117 TestHFileBlock.writeTestBlockContents(dos);
118 byte[] headerAndData = hbw.getHeaderAndData();
119 assertEquals(1000 * 4, hbw.getUncompressedSizeWithoutHeader());
120 hbw.releaseCompressor();
121 return hbw;
122 }
123
124 private String createTestBlockStr(Compression.Algorithm algo,
125 int correctLength) throws IOException {
126 Writer hbw = createTestV2Block(algo);
127 byte[] testV2Block = hbw.getHeaderAndData();
128 int osOffset = HFileBlock.HEADER_SIZE_NO_CHECKSUM + 9;
129 if (testV2Block.length == correctLength) {
130
131
132
133 testV2Block[osOffset] = 3;
134 }
135 return Bytes.toStringBinary(testV2Block);
136 }
137
138 @Test
139 public void testNoCompression() throws IOException {
140 assertEquals(4000, createTestV2Block(NONE).getBlockForCaching().
141 getUncompressedSizeWithoutHeader());
142 }
143
144 @Test
145 public void testGzipCompression() throws IOException {
146 final String correctTestBlockStr =
147 "DATABLK*\\x00\\x00\\x00:\\x00\\x00\\x0F\\xA0\\xFF\\xFF\\xFF\\xFF"
148 + "\\xFF\\xFF\\xFF\\xFF"
149
150 + "\\x1F\\x8B"
151 + "\\x08"
152 + "\\x00"
153 + "\\x00\\x00\\x00\\x00"
154 + "\\x00"
155
156
157 + "\\x03"
158 + "\\xED\\xC3\\xC1\\x11\\x00 \\x08\\xC00DD\\xDD\\x7Fa"
159 + "\\xD6\\xE8\\xA3\\xB9K\\x84`\\x96Q\\xD3\\xA8\\xDB\\xA8e\\xD4c"
160 + "\\xD46\\xEA5\\xEA3\\xEA7\\xE7\\x00LI\\x5Cs\\xA0\\x0F\\x00\\x00";
161 final int correctGzipBlockLength = 82;
162 assertEquals(correctTestBlockStr, createTestBlockStr(GZ,
163 correctGzipBlockLength));
164 }
165
166 @Test
167 public void testReaderV1() throws IOException {
168 for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
169 for (boolean pread : new boolean[] { false, true }) {
170 byte[] block = createTestV1Block(algo);
171 Path path = new Path(TEST_UTIL.getDataTestDir(),
172 "blocks_v1_"+ algo);
173 LOG.info("Creating temporary file at " + path);
174 FSDataOutputStream os = fs.create(path);
175 int totalSize = 0;
176 int numBlocks = 50;
177 for (int i = 0; i < numBlocks; ++i) {
178 os.write(block);
179 totalSize += block.length;
180 }
181 os.close();
182
183 FSDataInputStream is = fs.open(path);
184 HFileBlock.FSReader hbr = new HFileBlock.FSReaderV1(is, algo,
185 totalSize);
186 HFileBlock b;
187 int numBlocksRead = 0;
188 long pos = 0;
189 while (pos < totalSize) {
190 b = hbr.readBlockData(pos, block.length, uncompressedSizeV1, pread);
191 b.sanityCheck();
192 pos += block.length;
193 numBlocksRead++;
194 }
195 assertEquals(numBlocks, numBlocksRead);
196 is.close();
197 }
198 }
199 }
200
201 @Test
202 public void testReaderV2() throws IOException {
203 for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
204 for (boolean pread : new boolean[] { false, true }) {
205 LOG.info("testReaderV2: Compression algorithm: " + algo +
206 ", pread=" + pread);
207 Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
208 + algo);
209 FSDataOutputStream os = fs.create(path);
210 Writer hbw = new Writer(algo, null,
211 includesMemstoreTS);
212 long totalSize = 0;
213 for (int blockId = 0; blockId < 2; ++blockId) {
214 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
215 for (int i = 0; i < 1234; ++i)
216 dos.writeInt(i);
217 hbw.writeHeaderAndData(os);
218 totalSize += hbw.getOnDiskSizeWithHeader();
219 }
220 os.close();
221
222 FSDataInputStream is = fs.open(path);
223 HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, is, algo,
224 totalSize, MINOR_VERSION, fs, path);
225 HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
226 is.close();
227
228 b.sanityCheck();
229 assertEquals(4936, b.getUncompressedSizeWithoutHeader());
230 assertEquals(algo == GZ ? 2173 : 4936,
231 b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
232 String blockStr = b.toString();
233
234 if (algo == GZ) {
235 is = fs.open(path);
236 hbr = new HFileBlock.FSReaderV2(is, is, algo, totalSize, MINOR_VERSION,
237 fs, path);
238 b = hbr.readBlockData(0, 2173 + HFileBlock.HEADER_SIZE_NO_CHECKSUM +
239 b.totalChecksumBytes(), -1, pread);
240 assertEquals(blockStr, b.toString());
241 int wrongCompressedSize = 2172;
242 try {
243 b = hbr.readBlockData(0, wrongCompressedSize
244 + HFileBlock.HEADER_SIZE_NO_CHECKSUM, -1, pread);
245 fail("Exception expected");
246 } catch (IOException ex) {
247 String expectedPrefix = "On-disk size without header provided is "
248 + wrongCompressedSize + ", but block header contains "
249 + b.getOnDiskSizeWithoutHeader() + ".";
250 assertTrue("Invalid exception message: '" + ex.getMessage()
251 + "'.\nMessage is expected to start with: '" + expectedPrefix
252 + "'", ex.getMessage().startsWith(expectedPrefix));
253 }
254 is.close();
255 }
256 }
257 }
258 }
259
260
261
262
263
264 @Test
265 public void testDataBlockEncoding() throws IOException {
266 final int numBlocks = 5;
267 for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
268 for (boolean pread : new boolean[] { false, true }) {
269 for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
270 LOG.info("testDataBlockEncoding algo " + algo +
271 " pread = " + pread +
272 " encoding " + encoding);
273 Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
274 + algo + "_" + encoding.toString());
275 FSDataOutputStream os = fs.create(path);
276 HFileDataBlockEncoder dataBlockEncoder =
277 new HFileDataBlockEncoderImpl(encoding);
278 Writer hbw = new Writer(algo, dataBlockEncoder,
279 includesMemstoreTS);
280 long totalSize = 0;
281 final List<Integer> encodedSizes = new ArrayList<Integer>();
282 final List<ByteBuffer> encodedBlocks = new ArrayList<ByteBuffer>();
283 for (int blockId = 0; blockId < numBlocks; ++blockId) {
284 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
285 TestHFileBlock.writeEncodedBlock(encoding, dos, encodedSizes, encodedBlocks,
286 blockId, includesMemstoreTS);
287
288 hbw.writeHeaderAndData(os);
289 totalSize += hbw.getOnDiskSizeWithHeader();
290 }
291 os.close();
292
293 FSDataInputStream is = fs.open(path);
294 HFileBlock.FSReaderV2 hbr = new HFileBlock.FSReaderV2(is, is, algo,
295 totalSize, MINOR_VERSION, fs, path);
296 hbr.setDataBlockEncoder(dataBlockEncoder);
297 hbr.setIncludesMemstoreTS(includesMemstoreTS);
298
299 HFileBlock b;
300 int pos = 0;
301 for (int blockId = 0; blockId < numBlocks; ++blockId) {
302 b = hbr.readBlockData(pos, -1, -1, pread);
303 b.sanityCheck();
304 pos += b.getOnDiskSizeWithHeader();
305
306 assertEquals((int) encodedSizes.get(blockId),
307 b.getUncompressedSizeWithoutHeader());
308 ByteBuffer actualBuffer = b.getBufferWithoutHeader();
309 if (encoding != DataBlockEncoding.NONE) {
310
311 assertEquals(0, actualBuffer.get(0));
312 assertEquals(encoding.getId(), actualBuffer.get(1));
313 actualBuffer.position(2);
314 actualBuffer = actualBuffer.slice();
315 }
316
317 ByteBuffer expectedBuffer = encodedBlocks.get(blockId);
318 expectedBuffer.rewind();
319
320
321 TestHFileBlock.assertBuffersEqual(expectedBuffer, actualBuffer, algo, encoding,
322 pread);
323 }
324 is.close();
325 }
326 }
327 }
328 }
329
330 @org.junit.Rule
331 public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
332 new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
333
334
335
336
337
338
339
340
341
342
343
344 public static final class Writer {
345
346
347 private static final int HEADER_SIZE = HFileBlock.HEADER_SIZE_NO_CHECKSUM;
348 private static final boolean DONT_FILL_HEADER = HFileBlock.DONT_FILL_HEADER;
349 private static final byte[] DUMMY_HEADER =
350 HFileBlock.DUMMY_HEADER_NO_CHECKSUM;
351
352 private enum State {
353 INIT,
354 WRITING,
355 BLOCK_READY
356 };
357
358
359 private State state = State.INIT;
360
361
362 private final Compression.Algorithm compressAlgo;
363
364
365 private final HFileDataBlockEncoder dataBlockEncoder;
366
367
368
369
370
371
372
373 private ByteArrayOutputStream baosInMemory;
374
375
376 private Compressor compressor;
377
378
379 private CompressionOutputStream compressionStream;
380
381
382 private ByteArrayOutputStream compressedByteStream;
383
384
385
386
387
388
389 private BlockType blockType;
390
391
392
393
394
395 private DataOutputStream userDataStream;
396
397
398
399
400
401 private byte[] onDiskBytesWithHeader;
402
403
404
405
406
407
408 private byte[] uncompressedBytesWithHeader;
409
410
411
412
413
414 private long startOffset;
415
416
417
418
419
420 private long[] prevOffsetByType;
421
422
423 private long prevOffset;
424
425
426 private boolean includesMemstoreTS;
427
428
429
430
431
432 public Writer(Compression.Algorithm compressionAlgorithm,
433 HFileDataBlockEncoder dataBlockEncoder, boolean includesMemstoreTS) {
434 compressAlgo = compressionAlgorithm == null ? NONE : compressionAlgorithm;
435 this.dataBlockEncoder = dataBlockEncoder != null
436 ? dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
437
438 baosInMemory = new ByteArrayOutputStream();
439 if (compressAlgo != NONE) {
440 compressor = compressionAlgorithm.getCompressor();
441 compressedByteStream = new ByteArrayOutputStream();
442 try {
443 compressionStream =
444 compressionAlgorithm.createPlainCompressionStream(
445 compressedByteStream, compressor);
446 } catch (IOException e) {
447 throw new RuntimeException("Could not create compression stream " +
448 "for algorithm " + compressionAlgorithm, e);
449 }
450 }
451
452 prevOffsetByType = new long[BlockType.values().length];
453 for (int i = 0; i < prevOffsetByType.length; ++i)
454 prevOffsetByType[i] = -1;
455
456 this.includesMemstoreTS = includesMemstoreTS;
457 }
458
459
460
461
462
463
464
465 public DataOutputStream startWriting(BlockType newBlockType)
466 throws IOException {
467 if (state == State.BLOCK_READY && startOffset != -1) {
468
469
470 prevOffsetByType[blockType.getId()] = startOffset;
471 }
472
473 startOffset = -1;
474 blockType = newBlockType;
475
476 baosInMemory.reset();
477 baosInMemory.write(DUMMY_HEADER);
478
479 state = State.WRITING;
480
481
482 userDataStream = new DataOutputStream(baosInMemory);
483 return userDataStream;
484 }
485
486
487
488
489
490
491
492
493 DataOutputStream getUserDataStream() {
494 expectState(State.WRITING);
495 return userDataStream;
496 }
497
498
499
500
501
502 private void ensureBlockReady() throws IOException {
503 Preconditions.checkState(state != State.INIT,
504 "Unexpected state: " + state);
505
506 if (state == State.BLOCK_READY)
507 return;
508
509
510 finishBlock();
511 }
512
513
514
515
516
517
518
519 private void finishBlock() throws IOException {
520 userDataStream.flush();
521
522
523 uncompressedBytesWithHeader = baosInMemory.toByteArray();
524 LOG.warn("Writer.finishBlock user data size with header before compression " +
525 uncompressedBytesWithHeader.length);
526 prevOffset = prevOffsetByType[blockType.getId()];
527
528
529
530
531 state = State.BLOCK_READY;
532 encodeDataBlockForDisk();
533
534 doCompression();
535 putHeader(uncompressedBytesWithHeader, 0, onDiskBytesWithHeader.length,
536 uncompressedBytesWithHeader.length);
537 }
538
539
540
541
542
543 private void doCompression() throws IOException {
544
545 if (compressAlgo != NONE) {
546 compressedByteStream.reset();
547 compressedByteStream.write(DUMMY_HEADER);
548
549 compressionStream.resetState();
550
551 compressionStream.write(uncompressedBytesWithHeader, HEADER_SIZE,
552 uncompressedBytesWithHeader.length - HEADER_SIZE);
553
554 compressionStream.flush();
555 compressionStream.finish();
556
557 onDiskBytesWithHeader = compressedByteStream.toByteArray();
558 putHeader(onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length,
559 uncompressedBytesWithHeader.length);
560 } else {
561 onDiskBytesWithHeader = uncompressedBytesWithHeader;
562 }
563 }
564
565
566
567
568
569 private void encodeDataBlockForDisk() throws IOException {
570 if (blockType != BlockType.DATA) {
571 return;
572 }
573
574
575 ByteBuffer rawKeyValues = ByteBuffer.wrap(uncompressedBytesWithHeader,
576 HEADER_SIZE, uncompressedBytesWithHeader.length -
577 HEADER_SIZE).slice();
578 Pair<ByteBuffer, BlockType> encodingResult =
579 dataBlockEncoder.beforeWriteToDisk(rawKeyValues,
580 includesMemstoreTS, DUMMY_HEADER);
581
582 BlockType encodedBlockType = encodingResult.getSecond();
583 if (encodedBlockType == BlockType.ENCODED_DATA) {
584 uncompressedBytesWithHeader = encodingResult.getFirst().array();
585 blockType = BlockType.ENCODED_DATA;
586 } else {
587
588 if (encodedBlockType != BlockType.DATA) {
589 throw new IOException("Unexpected block type coming out of data " +
590 "block encoder: " + encodedBlockType);
591 }
592 if (userDataStream.size() !=
593 uncompressedBytesWithHeader.length - HEADER_SIZE) {
594 throw new IOException("Uncompressed size mismatch: "
595 + userDataStream.size() + " vs. "
596 + (uncompressedBytesWithHeader.length - HEADER_SIZE));
597 }
598 }
599 }
600
601
602
603
604
605
606
607 private void putHeader(byte[] dest, int offset, int onDiskSize,
608 int uncompressedSize) {
609 offset = blockType.put(dest, offset);
610 offset = Bytes.putInt(dest, offset, onDiskSize - HEADER_SIZE);
611 offset = Bytes.putInt(dest, offset, uncompressedSize - HEADER_SIZE);
612 Bytes.putLong(dest, offset, prevOffset);
613 }
614
615
616
617
618
619
620
621
622
623 public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
624 long offset = out.getPos();
625 if (startOffset != -1 && offset != startOffset) {
626 throw new IOException("A " + blockType + " block written to a "
627 + "stream twice, first at offset " + startOffset + ", then at "
628 + offset);
629 }
630 startOffset = offset;
631
632 writeHeaderAndData((DataOutputStream) out);
633 }
634
635
636
637
638
639
640
641
642
643
644 private void writeHeaderAndData(DataOutputStream out) throws IOException {
645 ensureBlockReady();
646 out.write(onDiskBytesWithHeader);
647 }
648
649
650
651
652
653
654
655
656
657
658 public byte[] getHeaderAndData() throws IOException {
659 ensureBlockReady();
660 return onDiskBytesWithHeader;
661 }
662
663
664
665
666
667 public void releaseCompressor() {
668 if (compressor != null) {
669 compressAlgo.returnCompressor(compressor);
670 compressor = null;
671 }
672 }
673
674
675
676
677
678
679
680
681
682 public int getOnDiskSizeWithoutHeader() {
683 expectState(State.BLOCK_READY);
684 return onDiskBytesWithHeader.length - HEADER_SIZE;
685 }
686
687
688
689
690
691
692
693
694 public int getOnDiskSizeWithHeader() {
695 expectState(State.BLOCK_READY);
696 return onDiskBytesWithHeader.length;
697 }
698
699
700
701
702 public int getUncompressedSizeWithoutHeader() {
703 expectState(State.BLOCK_READY);
704 return uncompressedBytesWithHeader.length - HEADER_SIZE;
705 }
706
707
708
709
710 public int getUncompressedSizeWithHeader() {
711 expectState(State.BLOCK_READY);
712 return uncompressedBytesWithHeader.length;
713 }
714
715
716 public boolean isWriting() {
717 return state == State.WRITING;
718 }
719
720
721
722
723
724
725
726
727 public int blockSizeWritten() {
728 if (state != State.WRITING)
729 return 0;
730 return userDataStream.size();
731 }
732
733
734
735
736
737
738
739
740 private byte[] getUncompressedDataWithHeader() {
741 expectState(State.BLOCK_READY);
742
743 return uncompressedBytesWithHeader;
744 }
745
746 private void expectState(State expectedState) {
747 if (state != expectedState) {
748 throw new IllegalStateException("Expected state: " + expectedState +
749 ", actual state: " + state);
750 }
751 }
752
753
754
755
756
757
758
759 public ByteBuffer getUncompressedBufferWithHeader() {
760 byte[] b = getUncompressedDataWithHeader();
761 return ByteBuffer.wrap(b, 0, b.length);
762 }
763
764
765
766
767
768
769
770
771
772
773
774 public void writeBlock(BlockWritable bw, FSDataOutputStream out)
775 throws IOException {
776 bw.writeToBlock(startWriting(bw.getBlockType()));
777 writeHeaderAndData(out);
778 }
779
780
781
782
783 public HFileBlock getBlockForCaching() {
784 return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
785 getUncompressedSizeWithoutHeader(), prevOffset,
786 getUncompressedBufferWithHeader(), DONT_FILL_HEADER, startOffset,
787 includesMemstoreTS, MINOR_VERSION, 0, ChecksumType.NULL.getCode(),
788 getOnDiskSizeWithoutHeader());
789 }
790 }
791 }
792