1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.io.hfile;
22  
23  import java.io.ByteArrayOutputStream;
24  import java.io.DataOutputStream;
25  import java.io.IOException;
26  import java.nio.ByteBuffer;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.Collection;
30  import java.util.HashSet;
31  import java.util.List;
32  import java.util.Random;
33  import java.util.Set;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FSDataInputStream;
39  import org.apache.hadoop.fs.FSDataOutputStream;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.HBaseTestingUtility;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
45  import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.ClassSize;
48  
49  import org.junit.Before;
50  import org.junit.Test;
51  import org.junit.runner.RunWith;
52  import org.junit.runners.Parameterized;
53  import org.junit.runners.Parameterized.Parameters;
54  
55  import static org.junit.Assert.*;
56  
57  @RunWith(Parameterized.class)
58  public class TestHFileBlockIndex {
59  
60    @Parameters
61    public static Collection<Object[]> compressionAlgorithms() {
62      return HBaseTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED;
63    }
64  
65    public TestHFileBlockIndex(Compression.Algorithm compr) {
66      this.compr = compr;
67    }
68  
69    private static final Log LOG = LogFactory.getLog(TestHFileBlockIndex.class);
70  
71    private static final int NUM_DATA_BLOCKS = 1000;
72    private static final HBaseTestingUtility TEST_UTIL =
73        new HBaseTestingUtility();
74  
75    private static final int SMALL_BLOCK_SIZE = 4096;
76    private static final int NUM_KV = 10000;
77  
78    private static FileSystem fs;
79    private Path path;
80    private Random rand;
81    private long rootIndexOffset;
82    private int numRootEntries;
83    private int numLevels;
84    private static final List<byte[]> keys = new ArrayList<byte[]>();
85    private final Compression.Algorithm compr;
86    private byte[] firstKeyInFile;
87    private Configuration conf;
88  
89    private static final int[] INDEX_CHUNK_SIZES = { 4096, 512, 384 };
90    private static final int[] EXPECTED_NUM_LEVELS = { 2, 3, 4 };
91    private static final int[] UNCOMPRESSED_INDEX_SIZES =
92        { 19187, 21813, 23086 };
93  
94    static {
95      assert INDEX_CHUNK_SIZES.length == EXPECTED_NUM_LEVELS.length;
96      assert INDEX_CHUNK_SIZES.length == UNCOMPRESSED_INDEX_SIZES.length;
97    }
98  
99    @Before
100   public void setUp() throws IOException {
101     keys.clear();
102     rand = new Random(2389757);
103     firstKeyInFile = null;
104     conf = TEST_UTIL.getConfiguration();
105 
106     // This test requires at least HFile format version 2.
107     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
108 
109     fs = FileSystem.get(conf);
110   }
111 
112   @Test
113   public void testBlockIndex() throws IOException {
114     path = new Path(TEST_UTIL.getDataTestDir(), "block_index_" + compr);
115     writeWholeIndex();
116     readIndex();
117   }
118 
119   /**
120    * A wrapper around a block reader which only caches the results of the last
121    * operation. Not thread-safe.
122    */
123   private static class BlockReaderWrapper implements HFile.CachingBlockReader {
124 
125     private HFileBlock.FSReader realReader;
126     private long prevOffset;
127     private long prevOnDiskSize;
128     private boolean prevPread;
129     private HFileBlock prevBlock;
130 
131     public int hitCount = 0;
132     public int missCount = 0;
133 
134     public BlockReaderWrapper(HFileBlock.FSReader realReader) {
135       this.realReader = realReader;
136     }
137 
138     @Override
139     public HFileBlock readBlock(long offset, long onDiskSize,
140         boolean cacheBlock, boolean pread, boolean isCompaction)
141         throws IOException {
142       if (offset == prevOffset && onDiskSize == prevOnDiskSize &&
143           pread == prevPread) {
144         hitCount += 1;
145         return prevBlock;
146       }
147 
148       missCount += 1;
149       prevBlock = realReader.readBlockData(offset, onDiskSize,
150           -1, pread);
151       prevOffset = offset;
152       prevOnDiskSize = onDiskSize;
153       prevPread = pread;
154 
155       return prevBlock;
156     }
157   }
158 
159   public void readIndex() throws IOException {
160     long fileSize = fs.getFileStatus(path).getLen();
161     LOG.info("Size of " + path + ": " + fileSize);
162 
163     FSDataInputStream istream = fs.open(path);
164     HFileBlock.FSReader blockReader = new HFileBlock.FSReaderV2(istream,
165         compr, fs.getFileStatus(path).getLen());
166 
167     BlockReaderWrapper brw = new BlockReaderWrapper(blockReader);
168     HFileBlockIndex.BlockIndexReader indexReader =
169         new HFileBlockIndex.BlockIndexReader(
170             Bytes.BYTES_RAWCOMPARATOR, numLevels, brw);
171 
172     indexReader.readRootIndex(blockReader.blockRange(rootIndexOffset,
173         fileSize).nextBlockAsStream(BlockType.ROOT_INDEX), numRootEntries);
174 
175     long prevOffset = -1;
176     int i = 0;
177     int expectedHitCount = 0;
178     int expectedMissCount = 0;
179     LOG.info("Total number of keys: " + keys.size());
180     for (byte[] key : keys) {
181       assertTrue(key != null);
182       assertTrue(indexReader != null);
183       HFileBlock b = indexReader.seekToDataBlock(key, 0, key.length, null,
184           true, true, false);
185       if (Bytes.BYTES_RAWCOMPARATOR.compare(key, firstKeyInFile) < 0) {
186         assertTrue(b == null);
187         ++i;
188         continue;
189       }
190 
191       String keyStr = "key #" + i + ", " + Bytes.toStringBinary(key);
192 
193       assertTrue("seekToDataBlock failed for " + keyStr, b != null);
194 
195       if (prevOffset == b.getOffset()) {
196         assertEquals(++expectedHitCount, brw.hitCount);
197       } else {
198         LOG.info("First key in a new block: " + keyStr + ", block offset: "
199             + b.getOffset() + ")");
200         assertTrue(b.getOffset() > prevOffset);
201         assertEquals(++expectedMissCount, brw.missCount);
202         prevOffset = b.getOffset();
203       }
204       ++i;
205     }
206 
207     istream.close();
208   }
209 
210   private void writeWholeIndex() throws IOException {
211     assertEquals(0, keys.size());
212     HFileBlock.Writer hbw = new HFileBlock.Writer(compr);
213     FSDataOutputStream outputStream = fs.create(path);
214     HFileBlockIndex.BlockIndexWriter biw =
215         new HFileBlockIndex.BlockIndexWriter(hbw, null, null);
216 
217     for (int i = 0; i < NUM_DATA_BLOCKS; ++i) {
218       hbw.startWriting(BlockType.DATA, false).write(
219           String.valueOf(rand.nextInt(1000)).getBytes());
220       long blockOffset = outputStream.getPos();
221       hbw.writeHeaderAndData(outputStream);
222 
223       byte[] firstKey = null;
224       for (int j = 0; j < 16; ++j) {
225         byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 16 + j);
226         keys.add(k);
227         if (j == 8)
228           firstKey = k;
229       }
230       assertTrue(firstKey != null);
231       if (firstKeyInFile == null)
232         firstKeyInFile = firstKey;
233       biw.addEntry(firstKey, blockOffset, hbw.getOnDiskSizeWithHeader());
234 
235       writeInlineBlocks(hbw, outputStream, biw, false);
236     }
237     writeInlineBlocks(hbw, outputStream, biw, true);
238     rootIndexOffset = biw.writeIndexBlocks(outputStream);
239     outputStream.close();
240 
241     numLevels = biw.getNumLevels();
242     numRootEntries = biw.getNumRootEntries();
243 
244     LOG.info("Index written: numLevels=" + numLevels + ", numRootEntries=" +
245         numRootEntries + ", rootIndexOffset=" + rootIndexOffset);
246   }
247 
248   private void writeInlineBlocks(HFileBlock.Writer hbw,
249       FSDataOutputStream outputStream, HFileBlockIndex.BlockIndexWriter biw,
250       boolean isClosing) throws IOException {
251     while (biw.shouldWriteBlock(isClosing)) {
252       long offset = outputStream.getPos();
253       biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType(), false));
254       hbw.writeHeaderAndData(outputStream);
255       biw.blockWritten(offset, hbw.getOnDiskSizeWithHeader(),
256           hbw.getUncompressedSizeWithoutHeader());
257       LOG.info("Wrote an inline index block at " + offset + ", size " +
258           hbw.getOnDiskSizeWithHeader());
259     }
260   }
261 
262   private static final long getDummyFileOffset(int i) {
263     return i * 185 + 379;
264   }
265 
266   private static final int getDummyOnDiskSize(int i) {
267     return i * i * 37 + i * 19 + 13;
268   }
269 
270   @Test
271   public void testSecondaryIndexBinarySearch() throws IOException {
272     int numTotalKeys = 99;
273     assertTrue(numTotalKeys % 2 == 1); // Ensure no one made this even.
274 
275     // We only add odd-index keys into the array that we will binary-search.
276     int numSearchedKeys = (numTotalKeys - 1) / 2;
277 
278     ByteArrayOutputStream baos = new ByteArrayOutputStream();
279     DataOutputStream dos = new DataOutputStream(baos);
280 
281     dos.writeInt(numSearchedKeys);
282     int curAllEntriesSize = 0;
283     int numEntriesAdded = 0;
284 
285     // Only odd-index elements of this array are used to keep the secondary
286     // index entries of the corresponding keys.
287     int secondaryIndexEntries[] = new int[numTotalKeys];
288 
289     for (int i = 0; i < numTotalKeys; ++i) {
290       byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 2);
291       keys.add(k);
292       String msgPrefix = "Key #" + i + " (" + Bytes.toStringBinary(k) + "): ";
293       StringBuilder padding = new StringBuilder();
294       while (msgPrefix.length() + padding.length() < 70)
295         padding.append(' ');
296       msgPrefix += padding;
297       if (i % 2 == 1) {
298         dos.writeInt(curAllEntriesSize);
299         secondaryIndexEntries[i] = curAllEntriesSize;
300         LOG.info(msgPrefix + "secondary index entry #" + ((i - 1) / 2) +
301             ", offset " + curAllEntriesSize);
302         curAllEntriesSize += k.length
303             + HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD;
304         ++numEntriesAdded;
305       } else {
306         secondaryIndexEntries[i] = -1;
307         LOG.info(msgPrefix + "not in the searched array");
308       }
309     }
310 
311     // Make sure the keys are increasing.
312     for (int i = 0; i < keys.size() - 1; ++i)
313       assertTrue(Bytes.BYTES_RAWCOMPARATOR.compare(keys.get(i),
314           keys.get(i + 1)) < 0);
315 
316     dos.writeInt(curAllEntriesSize);
317     assertEquals(numSearchedKeys, numEntriesAdded);
318     int secondaryIndexOffset = dos.size();
319     assertEquals(Bytes.SIZEOF_INT * (numSearchedKeys + 2),
320         secondaryIndexOffset);
321 
322     for (int i = 1; i <= numTotalKeys - 1; i += 2) {
323       assertEquals(dos.size(),
324           secondaryIndexOffset + secondaryIndexEntries[i]);
325       long dummyFileOffset = getDummyFileOffset(i);
326       int dummyOnDiskSize = getDummyOnDiskSize(i);
327       LOG.debug("Storing file offset=" + dummyFileOffset + " and onDiskSize=" +
328           dummyOnDiskSize + " at offset " + dos.size());
329       dos.writeLong(dummyFileOffset);
330       dos.writeInt(dummyOnDiskSize);
331       LOG.debug("Stored key " + ((i - 1) / 2) +" at offset " + dos.size());
332       dos.write(keys.get(i));
333     }
334 
335     dos.writeInt(curAllEntriesSize);
336 
337     ByteBuffer nonRootIndex = ByteBuffer.wrap(baos.toByteArray());
338     for (int i = 0; i < numTotalKeys; ++i) {
339       byte[] searchKey = keys.get(i);
340       byte[] arrayHoldingKey = new byte[searchKey.length +
341                                         searchKey.length / 2];
342 
343       // To make things a bit more interesting, store the key we are looking
344       // for at a non-zero offset in a new array.
345       System.arraycopy(searchKey, 0, arrayHoldingKey, searchKey.length / 2,
346             searchKey.length);
347 
348       int searchResult = BlockIndexReader.binarySearchNonRootIndex(
349           arrayHoldingKey, searchKey.length / 2, searchKey.length, nonRootIndex,
350           Bytes.BYTES_RAWCOMPARATOR);
351       String lookupFailureMsg = "Failed to look up key #" + i + " ("
352           + Bytes.toStringBinary(searchKey) + ")";
353 
354       int expectedResult;
355       int referenceItem;
356 
357       if (i % 2 == 1) {
358         // This key is in the array we search as the element (i - 1) / 2. Make
359         // sure we find it.
360         expectedResult = (i - 1) / 2;
361         referenceItem = i;
362       } else {
363         // This key is not in the array but between two elements on the array,
364         // in the beginning, or in the end. The result should be the previous
365         // key in the searched array, or -1 for i = 0.
366         expectedResult = i / 2 - 1;
367         referenceItem = i - 1;
368       }
369 
370       assertEquals(lookupFailureMsg, expectedResult, searchResult);
371 
372       // Now test we can get the offset and the on-disk-size using a
373       // higher-level API function.s
374       boolean locateBlockResult =
375         BlockIndexReader.locateNonRootIndexEntry(nonRootIndex, arrayHoldingKey,
376             searchKey.length / 2, searchKey.length, Bytes.BYTES_RAWCOMPARATOR);
377 
378       if (i == 0) {
379         assertFalse(locateBlockResult);
380       } else {
381         assertTrue(locateBlockResult);
382         String errorMsg = "i=" + i + ", position=" + nonRootIndex.position();
383         assertEquals(errorMsg, getDummyFileOffset(referenceItem),
384             nonRootIndex.getLong());
385         assertEquals(errorMsg, getDummyOnDiskSize(referenceItem),
386             nonRootIndex.getInt());
387       }
388     }
389 
390   }
391 
392   @Test
393   public void testBlockIndexChunk() throws IOException {
394     BlockIndexChunk c = new BlockIndexChunk();
395     ByteArrayOutputStream baos = new ByteArrayOutputStream();
396     int N = 1000;
397     int[] numSubEntriesAt = new int[N];
398     int numSubEntries = 0;
399     for (int i = 0; i < N; ++i) {
400       baos.reset();
401       DataOutputStream dos = new DataOutputStream(baos);
402       c.writeNonRoot(dos);
403       assertEquals(c.getNonRootSize(), dos.size());
404 
405       baos.reset();
406       dos = new DataOutputStream(baos);
407       c.writeRoot(dos);
408       assertEquals(c.getRootSize(), dos.size());
409 
410       byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i);
411       numSubEntries += rand.nextInt(5) + 1;
412       keys.add(k);
413       c.add(k, getDummyFileOffset(i), getDummyOnDiskSize(i), numSubEntries);
414     }
415 
416     // Test the ability to look up the entry that contains a particular
417     // deeper-level index block's entry ("sub-entry"), assuming a global
418     // 0-based ordering of sub-entries. This is needed for mid-key calculation.
419     for (int i = 0; i < N; ++i) {
420       for (int j = i == 0 ? 0 : numSubEntriesAt[i - 1];
421            j < numSubEntriesAt[i];
422            ++j) {
423         assertEquals(i, c.getEntryBySubEntry(j));
424       }
425     }
426   }
427 
428   /** Checks if the HeapSize calculator is within reason */
429   @Test
430   public void testHeapSizeForBlockIndex() throws IOException {
431     Class<HFileBlockIndex.BlockIndexReader> cl =
432         HFileBlockIndex.BlockIndexReader.class;
433     long expected = ClassSize.estimateBase(cl, false);
434 
435     HFileBlockIndex.BlockIndexReader bi =
436         new HFileBlockIndex.BlockIndexReader(Bytes.BYTES_RAWCOMPARATOR, 1);
437     long actual = bi.heapSize();
438 
439     // Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets,
440     // int [] blockDataSizes) are all null they are not going to show up in the
441     // HeapSize calculation, so need to remove those array costs from expected.
442     expected -= ClassSize.align(3 * ClassSize.ARRAY);
443 
444     if (expected != actual) {
445       ClassSize.estimateBase(cl, true);
446       assertEquals(expected, actual);
447     }
448   }
449 
450   /**
451    * Testing block index through the HFile writer/reader APIs. Allows to test
452    * setting index block size through configuration, intermediate-level index
453    * blocks, and caching index blocks on write.
454    *
455    * @throws IOException
456    */
457   @Test
458   public void testHFileWriterAndReader() throws IOException {
459     Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
460         "hfile_for_block_index");
461     CacheConfig cacheConf = new CacheConfig(conf);
462     BlockCache blockCache = cacheConf.getBlockCache();
463 
464     for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) {
465       int indexBlockSize = INDEX_CHUNK_SIZES[testI];
466       int expectedNumLevels = EXPECTED_NUM_LEVELS[testI];
467       LOG.info("Index block size: " + indexBlockSize + ", compression: "
468           + compr);
469       // Evict all blocks that were cached-on-write by the previous invocation.
470       blockCache.evictBlocksByPrefix(hfilePath.getName()
471           + HFile.CACHE_KEY_SEPARATOR);
472 
473       conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
474       Set<String> keyStrSet = new HashSet<String>();
475       byte[][] keys = new byte[NUM_KV][];
476       byte[][] values = new byte[NUM_KV][];
477 
478       // Write the HFile
479       {
480         HFile.Writer writer =
481           HFile.getWriterFactory(conf, cacheConf).createWriter(fs,
482             hfilePath, SMALL_BLOCK_SIZE, compr, KeyValue.KEY_COMPARATOR);
483         Random rand = new Random(19231737);
484 
485         for (int i = 0; i < NUM_KV; ++i) {
486           byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
487 
488           // Key will be interpreted by KeyValue.KEY_COMPARATOR
489           byte[] k = KeyValue.createFirstOnRow(row, 0, row.length, row, 0, 0,
490               row, 0, 0).getKey();
491 
492           byte[] v = TestHFileWriterV2.randomValue(rand);
493           writer.append(k, v);
494           keys[i] = k;
495           values[i] = v;
496           keyStrSet.add(Bytes.toStringBinary(k));
497 
498           if (i > 0) {
499             assertTrue(KeyValue.KEY_COMPARATOR.compare(keys[i - 1],
500                 keys[i]) < 0);
501           }
502         }
503 
504         writer.close();
505       }
506 
507       // Read the HFile
508       HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf);
509       assertEquals(expectedNumLevels,
510           reader.getTrailer().getNumDataIndexLevels());
511 
512       assertTrue(Bytes.equals(keys[0], reader.getFirstKey()));
513       assertTrue(Bytes.equals(keys[NUM_KV - 1], reader.getLastKey()));
514       LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1]));
515 
516       for (boolean pread : new boolean[] { false, true }) {
517         HFileScanner scanner = reader.getScanner(true, pread);
518         for (int i = 0; i < NUM_KV; ++i) {
519           checkSeekTo(keys, scanner, i);
520           checkKeyValue("i=" + i, keys[i], values[i], scanner.getKey(),
521               scanner.getValue());
522         }
523         assertTrue(scanner.seekTo());
524         for (int i = NUM_KV - 1; i >= 0; --i) {
525           checkSeekTo(keys, scanner, i);
526           checkKeyValue("i=" + i, keys[i], values[i], scanner.getKey(),
527               scanner.getValue());
528         }
529       }
530 
531       // Manually compute the mid-key and validate it.
532       HFileReaderV2 reader2 = (HFileReaderV2) reader;
533       HFileBlock.FSReader fsReader = reader2.getUncachedBlockReader();
534 
535       HFileBlock.BlockIterator iter = fsReader.blockRange(0,
536           reader.getTrailer().getLoadOnOpenDataOffset());
537       HFileBlock block;
538       List<byte[]> blockKeys = new ArrayList<byte[]>();
539       while ((block = iter.nextBlock()) != null) {
540         if (block.getBlockType() != BlockType.LEAF_INDEX)
541           return;
542         ByteBuffer b = block.getBufferReadOnly();
543         int n = b.getInt();
544         // One int for the number of items, and n + 1 for the secondary index.
545         int entriesOffset = Bytes.SIZEOF_INT * (n + 2);
546 
547         // Get all the keys from the leaf index block. S
548         for (int i = 0; i < n; ++i) {
549           int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (i + 1));
550           int nextKeyRelOffset = b.getInt(Bytes.SIZEOF_INT * (i + 2));
551           int keyLen = nextKeyRelOffset - keyRelOffset;
552           int keyOffset = b.arrayOffset() + entriesOffset + keyRelOffset +
553               HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD;
554           byte[] blockKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset
555               + keyLen);
556           String blockKeyStr = Bytes.toString(blockKey);
557           blockKeys.add(blockKey);
558 
559           // If the first key of the block is not among the keys written, we
560           // are not parsing the non-root index block format correctly.
561           assertTrue("Invalid block key from leaf-level block: " + blockKeyStr,
562               keyStrSet.contains(blockKeyStr));
563         }
564       }
565 
566       // Validate the mid-key.
567       assertEquals(
568           Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)),
569           Bytes.toStringBinary(reader.midkey()));
570 
571       assertEquals(UNCOMPRESSED_INDEX_SIZES[testI],
572           reader.getTrailer().getUncompressedDataIndexSize());
573 
574       reader.close();
575     }
576   }
577 
578   private void checkSeekTo(byte[][] keys, HFileScanner scanner, int i)
579       throws IOException {
580     assertEquals("Failed to seek to key #" + i + " ("
581         + Bytes.toStringBinary(keys[i]) + ")", 0, scanner.seekTo(keys[i]));
582   }
583 
584   private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr,
585       ByteBuffer buf) {
586     assertEquals(msgPrefix + ": expected " + Bytes.toStringBinary(arr)
587         + ", actual " + Bytes.toStringBinary(buf), 0, Bytes.compareTo(arr, 0,
588         arr.length, buf.array(), buf.arrayOffset(), buf.limit()));
589   }
590 
591   /** Check a key/value pair after it was read by the reader */
592   private void checkKeyValue(String msgPrefix, byte[] expectedKey,
593       byte[] expectedValue, ByteBuffer keyRead, ByteBuffer valueRead) {
594     if (!msgPrefix.isEmpty())
595       msgPrefix += ". ";
596 
597     assertArrayEqualsBuffer(msgPrefix + "Invalid key", expectedKey, keyRead);
598     assertArrayEqualsBuffer(msgPrefix + "Invalid value", expectedValue,
599         valueRead);
600   }
601 
602 }