001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.hfile;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertTrue;
023
024import java.io.ByteArrayOutputStream;
025import java.io.DataOutputStream;
026import java.io.IOException;
027import java.nio.ByteBuffer;
028import java.util.ArrayList;
029import java.util.Arrays;
030import java.util.Collection;
031import java.util.HashSet;
032import java.util.List;
033import java.util.Random;
034import java.util.Set;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.fs.FSDataInputStream;
037import org.apache.hadoop.fs.FSDataOutputStream;
038import org.apache.hadoop.fs.FileSystem;
039import org.apache.hadoop.fs.Path;
040import org.apache.hadoop.hbase.CellComparatorImpl;
041import org.apache.hadoop.hbase.CellUtil;
042import org.apache.hadoop.hbase.HBaseClassTestRule;
043import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
044import org.apache.hadoop.hbase.HBaseTestingUtility;
045import org.apache.hadoop.hbase.KeyValue;
046import org.apache.hadoop.hbase.KeyValueUtil;
047import org.apache.hadoop.hbase.PrivateCellUtil;
048import org.apache.hadoop.hbase.fs.HFileSystem;
049import org.apache.hadoop.hbase.io.compress.Compression;
050import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
051import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
052import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk;
053import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
054import org.apache.hadoop.hbase.nio.ByteBuff;
055import org.apache.hadoop.hbase.nio.MultiByteBuff;
056import org.apache.hadoop.hbase.testclassification.IOTests;
057import org.apache.hadoop.hbase.testclassification.MediumTests;
058import org.apache.hadoop.hbase.util.Bytes;
059import org.apache.hadoop.hbase.util.ClassSize;
060import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
061import org.junit.Before;
062import org.junit.ClassRule;
063import org.junit.Test;
064import org.junit.experimental.categories.Category;
065import org.junit.runner.RunWith;
066import org.junit.runners.Parameterized;
067import org.junit.runners.Parameterized.Parameters;
068import org.slf4j.Logger;
069import org.slf4j.LoggerFactory;
070
071@RunWith(Parameterized.class)
072@Category({IOTests.class, MediumTests.class})
073public class TestHFileBlockIndex {
074
075  @ClassRule
076  public static final HBaseClassTestRule CLASS_RULE =
077      HBaseClassTestRule.forClass(TestHFileBlockIndex.class);
078
079  @Parameters
080  public static Collection<Object[]> compressionAlgorithms() {
081    return HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS_PARAMETERIZED;
082  }
083
084  public TestHFileBlockIndex(Compression.Algorithm compr) {
085    this.compr = compr;
086  }
087
088  private static final Logger LOG = LoggerFactory.getLogger(TestHFileBlockIndex.class);
089
090  private static final int NUM_DATA_BLOCKS = 1000;
091  private static final HBaseTestingUtility TEST_UTIL =
092      new HBaseTestingUtility();
093
094  private static final int SMALL_BLOCK_SIZE = 4096;
095  private static final int NUM_KV = 10000;
096
097  private static FileSystem fs;
098  private Path path;
099  private Random rand;
100  private long rootIndexOffset;
101  private int numRootEntries;
102  private int numLevels;
103  private static final List<byte[]> keys = new ArrayList<>();
104  private final Compression.Algorithm compr;
105  private byte[] firstKeyInFile;
106  private Configuration conf;
107
108  private static final int[] INDEX_CHUNK_SIZES = { 4096, 512, 384 };
109  private static final int[] EXPECTED_NUM_LEVELS = { 2, 3, 4 };
110  private static final int[] UNCOMPRESSED_INDEX_SIZES =
111      { 19187, 21813, 23086 };
112
113  private static final boolean includesMemstoreTS = true;
114
115  static {
116    assert INDEX_CHUNK_SIZES.length == EXPECTED_NUM_LEVELS.length;
117    assert INDEX_CHUNK_SIZES.length == UNCOMPRESSED_INDEX_SIZES.length;
118  }
119
120  @Before
121  public void setUp() throws IOException {
122    keys.clear();
123    rand = new Random(2389757);
124    firstKeyInFile = null;
125    conf = TEST_UTIL.getConfiguration();
126
127    // This test requires at least HFile format version 2.
128    conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
129
130    fs = HFileSystem.get(conf);
131  }
132
133  @Test
134  public void testBlockIndex() throws IOException {
135    testBlockIndexInternals(false);
136    clear();
137    testBlockIndexInternals(true);
138  }
139
140  private void clear() throws IOException {
141    keys.clear();
142    rand = new Random(2389757);
143    firstKeyInFile = null;
144    conf = TEST_UTIL.getConfiguration();
145
146    // This test requires at least HFile format version 2.
147    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
148
149    fs = HFileSystem.get(conf);
150  }
151
152  private void testBlockIndexInternals(boolean useTags) throws IOException {
153    path = new Path(TEST_UTIL.getDataTestDir(), "block_index_" + compr + useTags);
154    writeWholeIndex(useTags);
155    readIndex(useTags);
156  }
157
158  /**
159   * A wrapper around a block reader which only caches the results of the last
160   * operation. Not thread-safe.
161   */
162  private static class BlockReaderWrapper implements HFile.CachingBlockReader {
163
164    private HFileBlock.FSReader realReader;
165    private long prevOffset;
166    private long prevOnDiskSize;
167    private boolean prevPread;
168    private HFileBlock prevBlock;
169
170    public int hitCount = 0;
171    public int missCount = 0;
172
173    public BlockReaderWrapper(HFileBlock.FSReader realReader) {
174      this.realReader = realReader;
175    }
176
177    @Override
178    public void returnBlock(HFileBlock block) {
179    }
180
181    @Override
182    public HFileBlock readBlock(long offset, long onDiskSize,
183        boolean cacheBlock, boolean pread, boolean isCompaction,
184        boolean updateCacheMetrics, BlockType expectedBlockType,
185        DataBlockEncoding expectedDataBlockEncoding)
186        throws IOException {
187      if (offset == prevOffset && onDiskSize == prevOnDiskSize &&
188          pread == prevPread) {
189        hitCount += 1;
190        return prevBlock;
191      }
192
193      missCount += 1;
194      prevBlock = realReader.readBlockData(offset, onDiskSize, pread, false);
195      prevOffset = offset;
196      prevOnDiskSize = onDiskSize;
197      prevPread = pread;
198
199      return prevBlock;
200    }
201  }
202
203  private void readIndex(boolean useTags) throws IOException {
204    long fileSize = fs.getFileStatus(path).getLen();
205    LOG.info("Size of {}: {} compression={}", path, fileSize, compr.toString());
206
207    FSDataInputStream istream = fs.open(path);
208    HFileContext meta = new HFileContextBuilder()
209                        .withHBaseCheckSum(true)
210                        .withIncludesMvcc(includesMemstoreTS)
211                        .withIncludesTags(useTags)
212                        .withCompression(compr)
213                        .build();
214    HFileBlock.FSReader blockReader = new HFileBlock.FSReaderImpl(istream, fs.getFileStatus(path)
215        .getLen(), meta);
216
217    BlockReaderWrapper brw = new BlockReaderWrapper(blockReader);
218    HFileBlockIndex.BlockIndexReader indexReader =
219        new HFileBlockIndex.CellBasedKeyBlockIndexReader(
220            CellComparatorImpl.COMPARATOR, numLevels, brw);
221
222    indexReader.readRootIndex(blockReader.blockRange(rootIndexOffset,
223        fileSize).nextBlockWithBlockType(BlockType.ROOT_INDEX), numRootEntries);
224
225    long prevOffset = -1;
226    int i = 0;
227    int expectedHitCount = 0;
228    int expectedMissCount = 0;
229    LOG.info("Total number of keys: " + keys.size());
230    for (byte[] key : keys) {
231      assertTrue(key != null);
232      assertTrue(indexReader != null);
233      KeyValue.KeyOnlyKeyValue keyOnlyKey = new KeyValue.KeyOnlyKeyValue(key, 0, key.length);
234      HFileBlock b =
235          indexReader.seekToDataBlock(keyOnlyKey, null, true,
236            true, false, null);
237      if (PrivateCellUtil.compare(CellComparatorImpl.COMPARATOR, keyOnlyKey, firstKeyInFile, 0,
238        firstKeyInFile.length) < 0) {
239        assertTrue(b == null);
240        ++i;
241        continue;
242      }
243
244      String keyStr = "key #" + i + ", " + Bytes.toStringBinary(key);
245
246      assertTrue("seekToDataBlock failed for " + keyStr, b != null);
247
248      if (prevOffset == b.getOffset()) {
249        assertEquals(++expectedHitCount, brw.hitCount);
250      } else {
251        LOG.info("First key in a new block: " + keyStr + ", block offset: "
252            + b.getOffset() + ")");
253        assertTrue(b.getOffset() > prevOffset);
254        assertEquals(++expectedMissCount, brw.missCount);
255        prevOffset = b.getOffset();
256      }
257      ++i;
258    }
259
260    istream.close();
261  }
262
263  private void writeWholeIndex(boolean useTags) throws IOException {
264    assertEquals(0, keys.size());
265    HFileContext meta = new HFileContextBuilder()
266                        .withHBaseCheckSum(true)
267                        .withIncludesMvcc(includesMemstoreTS)
268                        .withIncludesTags(useTags)
269                        .withCompression(compr)
270                        .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
271                        .build();
272    HFileBlock.Writer hbw = new HFileBlock.Writer(null,
273        meta);
274    FSDataOutputStream outputStream = fs.create(path);
275    HFileBlockIndex.BlockIndexWriter biw =
276        new HFileBlockIndex.BlockIndexWriter(hbw, null, null);
277
278    for (int i = 0; i < NUM_DATA_BLOCKS; ++i) {
279      hbw.startWriting(BlockType.DATA).write(String.valueOf(rand.nextInt(1000)).getBytes());
280      long blockOffset = outputStream.getPos();
281      hbw.writeHeaderAndData(outputStream);
282
283      byte[] firstKey = null;
284      byte[] family = Bytes.toBytes("f");
285      byte[] qualifier = Bytes.toBytes("q");
286      for (int j = 0; j < 16; ++j) {
287        byte[] k =
288            new KeyValue(RandomKeyValueUtil.randomOrderedKey(rand, i * 16 + j), family, qualifier,
289                EnvironmentEdgeManager.currentTime(), KeyValue.Type.Put).getKey();
290        keys.add(k);
291        if (j == 8) {
292          firstKey = k;
293        }
294      }
295      assertTrue(firstKey != null);
296      if (firstKeyInFile == null) {
297        firstKeyInFile = firstKey;
298      }
299      biw.addEntry(firstKey, blockOffset, hbw.getOnDiskSizeWithHeader());
300
301      writeInlineBlocks(hbw, outputStream, biw, false);
302    }
303    writeInlineBlocks(hbw, outputStream, biw, true);
304    rootIndexOffset = biw.writeIndexBlocks(outputStream);
305    outputStream.close();
306
307    numLevels = biw.getNumLevels();
308    numRootEntries = biw.getNumRootEntries();
309
310    LOG.info("Index written: numLevels=" + numLevels + ", numRootEntries=" +
311        numRootEntries + ", rootIndexOffset=" + rootIndexOffset);
312  }
313
314  private void writeInlineBlocks(HFileBlock.Writer hbw,
315      FSDataOutputStream outputStream, HFileBlockIndex.BlockIndexWriter biw,
316      boolean isClosing) throws IOException {
317    while (biw.shouldWriteBlock(isClosing)) {
318      long offset = outputStream.getPos();
319      biw.writeInlineBlock(hbw.startWriting(biw.getInlineBlockType()));
320      hbw.writeHeaderAndData(outputStream);
321      biw.blockWritten(offset, hbw.getOnDiskSizeWithHeader(),
322          hbw.getUncompressedSizeWithoutHeader());
323      LOG.info("Wrote an inline index block at " + offset + ", size " +
324          hbw.getOnDiskSizeWithHeader());
325    }
326  }
327
328  private static final long getDummyFileOffset(int i) {
329    return i * 185 + 379;
330  }
331
332  private static final int getDummyOnDiskSize(int i) {
333    return i * i * 37 + i * 19 + 13;
334  }
335
336  @Test
337  public void testSecondaryIndexBinarySearch() throws IOException {
338    int numTotalKeys = 99;
339    assertTrue(numTotalKeys % 2 == 1); // Ensure no one made this even.
340
341    // We only add odd-index keys into the array that we will binary-search.
342    int numSearchedKeys = (numTotalKeys - 1) / 2;
343
344    ByteArrayOutputStream baos = new ByteArrayOutputStream();
345    DataOutputStream dos = new DataOutputStream(baos);
346
347    dos.writeInt(numSearchedKeys);
348    int curAllEntriesSize = 0;
349    int numEntriesAdded = 0;
350
351    // Only odd-index elements of this array are used to keep the secondary
352    // index entries of the corresponding keys.
353    int secondaryIndexEntries[] = new int[numTotalKeys];
354
355    for (int i = 0; i < numTotalKeys; ++i) {
356      byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i * 2);
357      KeyValue cell = new KeyValue(k, Bytes.toBytes("f"), Bytes.toBytes("q"),
358          Bytes.toBytes("val"));
359      //KeyValue cell = new KeyValue.KeyOnlyKeyValue(k, 0, k.length);
360      keys.add(cell.getKey());
361      String msgPrefix = "Key #" + i + " (" + Bytes.toStringBinary(k) + "): ";
362      StringBuilder padding = new StringBuilder();
363      while (msgPrefix.length() + padding.length() < 70)
364        padding.append(' ');
365      msgPrefix += padding;
366      if (i % 2 == 1) {
367        dos.writeInt(curAllEntriesSize);
368        secondaryIndexEntries[i] = curAllEntriesSize;
369        LOG.info(msgPrefix + "secondary index entry #" + ((i - 1) / 2) +
370            ", offset " + curAllEntriesSize);
371        curAllEntriesSize += cell.getKey().length
372            + HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD;
373        ++numEntriesAdded;
374      } else {
375        secondaryIndexEntries[i] = -1;
376        LOG.info(msgPrefix + "not in the searched array");
377      }
378    }
379
380    // Make sure the keys are increasing.
381    for (int i = 0; i < keys.size() - 1; ++i)
382      assertTrue(CellComparatorImpl.COMPARATOR.compare(
383          new KeyValue.KeyOnlyKeyValue(keys.get(i), 0, keys.get(i).length),
384          new KeyValue.KeyOnlyKeyValue(keys.get(i + 1), 0, keys.get(i + 1).length)) < 0);
385
386    dos.writeInt(curAllEntriesSize);
387    assertEquals(numSearchedKeys, numEntriesAdded);
388    int secondaryIndexOffset = dos.size();
389    assertEquals(Bytes.SIZEOF_INT * (numSearchedKeys + 2),
390        secondaryIndexOffset);
391
392    for (int i = 1; i <= numTotalKeys - 1; i += 2) {
393      assertEquals(dos.size(),
394          secondaryIndexOffset + secondaryIndexEntries[i]);
395      long dummyFileOffset = getDummyFileOffset(i);
396      int dummyOnDiskSize = getDummyOnDiskSize(i);
397      LOG.debug("Storing file offset=" + dummyFileOffset + " and onDiskSize=" +
398          dummyOnDiskSize + " at offset " + dos.size());
399      dos.writeLong(dummyFileOffset);
400      dos.writeInt(dummyOnDiskSize);
401      LOG.debug("Stored key " + ((i - 1) / 2) +" at offset " + dos.size());
402      dos.write(keys.get(i));
403    }
404
405    dos.writeInt(curAllEntriesSize);
406
407    ByteBuffer nonRootIndex = ByteBuffer.wrap(baos.toByteArray());
408    for (int i = 0; i < numTotalKeys; ++i) {
409      byte[] searchKey = keys.get(i);
410      byte[] arrayHoldingKey = new byte[searchKey.length +
411                                        searchKey.length / 2];
412
413      // To make things a bit more interesting, store the key we are looking
414      // for at a non-zero offset in a new array.
415      System.arraycopy(searchKey, 0, arrayHoldingKey, searchKey.length / 2,
416            searchKey.length);
417
418      KeyValue.KeyOnlyKeyValue cell = new KeyValue.KeyOnlyKeyValue(
419          arrayHoldingKey, searchKey.length / 2, searchKey.length);
420      int searchResult = BlockIndexReader.binarySearchNonRootIndex(cell,
421          new MultiByteBuff(nonRootIndex), CellComparatorImpl.COMPARATOR);
422      String lookupFailureMsg = "Failed to look up key #" + i + " ("
423          + Bytes.toStringBinary(searchKey) + ")";
424
425      int expectedResult;
426      int referenceItem;
427
428      if (i % 2 == 1) {
429        // This key is in the array we search as the element (i - 1) / 2. Make
430        // sure we find it.
431        expectedResult = (i - 1) / 2;
432        referenceItem = i;
433      } else {
434        // This key is not in the array but between two elements on the array,
435        // in the beginning, or in the end. The result should be the previous
436        // key in the searched array, or -1 for i = 0.
437        expectedResult = i / 2 - 1;
438        referenceItem = i - 1;
439      }
440
441      assertEquals(lookupFailureMsg, expectedResult, searchResult);
442
443      // Now test we can get the offset and the on-disk-size using a
444      // higher-level API function.s
445      boolean locateBlockResult =
446          (BlockIndexReader.locateNonRootIndexEntry(new MultiByteBuff(nonRootIndex), cell,
447          CellComparatorImpl.COMPARATOR) != -1);
448
449      if (i == 0) {
450        assertFalse(locateBlockResult);
451      } else {
452        assertTrue(locateBlockResult);
453        String errorMsg = "i=" + i + ", position=" + nonRootIndex.position();
454        assertEquals(errorMsg, getDummyFileOffset(referenceItem),
455            nonRootIndex.getLong());
456        assertEquals(errorMsg, getDummyOnDiskSize(referenceItem),
457            nonRootIndex.getInt());
458      }
459    }
460
461  }
462
463  @Test
464  public void testBlockIndexChunk() throws IOException {
465    BlockIndexChunk c = new BlockIndexChunk();
466    ByteArrayOutputStream baos = new ByteArrayOutputStream();
467    int N = 1000;
468    int[] numSubEntriesAt = new int[N];
469    int numSubEntries = 0;
470    for (int i = 0; i < N; ++i) {
471      baos.reset();
472      DataOutputStream dos = new DataOutputStream(baos);
473      c.writeNonRoot(dos);
474      assertEquals(c.getNonRootSize(), dos.size());
475
476      baos.reset();
477      dos = new DataOutputStream(baos);
478      c.writeRoot(dos);
479      assertEquals(c.getRootSize(), dos.size());
480
481      byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i);
482      numSubEntries += rand.nextInt(5) + 1;
483      keys.add(k);
484      c.add(k, getDummyFileOffset(i), getDummyOnDiskSize(i), numSubEntries);
485    }
486
487    // Test the ability to look up the entry that contains a particular
488    // deeper-level index block's entry ("sub-entry"), assuming a global
489    // 0-based ordering of sub-entries. This is needed for mid-key calculation.
490    for (int i = 0; i < N; ++i) {
491      for (int j = i == 0 ? 0 : numSubEntriesAt[i - 1];
492           j < numSubEntriesAt[i];
493           ++j) {
494        assertEquals(i, c.getEntryBySubEntry(j));
495      }
496    }
497  }
498
499  /** Checks if the HeapSize calculator is within reason */
500  @Test
501  public void testHeapSizeForBlockIndex() throws IOException {
502    Class<HFileBlockIndex.BlockIndexReader> cl =
503        HFileBlockIndex.BlockIndexReader.class;
504    long expected = ClassSize.estimateBase(cl, false);
505
506    HFileBlockIndex.BlockIndexReader bi =
507        new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
508    long actual = bi.heapSize();
509
510    // Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets,
511    // int [] blockDataSizes) are all null they are not going to show up in the
512    // HeapSize calculation, so need to remove those array costs from expected.
513    // Already the block keys are not there in this case
514    expected -= ClassSize.align(2 * ClassSize.ARRAY);
515
516    if (expected != actual) {
517      expected = ClassSize.estimateBase(cl, true);
518      assertEquals(expected, actual);
519    }
520  }
521
522  /**
523  * to check if looks good when midKey on a leaf index block boundary
524  * @throws IOException
525  */
526  @Test
527  public void testMidKeyOnLeafIndexBlockBoundary() throws IOException {
528    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "hfile_for_midkey");
529    int maxChunkSize = 512;
530    conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize);
531    // should open hfile.block.index.cacheonwrite
532    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, true);
533    CacheConfig.instantiateBlockCache(conf);
534    CacheConfig cacheConf = new CacheConfig(conf);
535    BlockCache blockCache = cacheConf.getBlockCache();
536    // Evict all blocks that were cached-on-write by the previous invocation.
537    blockCache.evictBlocksByHfileName(hfilePath.getName());
538    // Write the HFile
539    HFileContext meta =
540        new HFileContextBuilder().withBlockSize(SMALL_BLOCK_SIZE).withCompression(Algorithm.NONE)
541            .withDataBlockEncoding(DataBlockEncoding.NONE).build();
542    HFile.Writer writer =
543        HFile.getWriterFactory(conf, cacheConf).withPath(fs, hfilePath).withFileContext(meta)
544            .create();
545    Random rand = new Random(19231737);
546    byte[] family = Bytes.toBytes("f");
547    byte[] qualifier = Bytes.toBytes("q");
548    int kvNumberToBeWritten = 16;
549    // the new generated hfile will contain 2 leaf-index blocks and 16 data blocks,
550    // midkey is just on the boundary of the first leaf-index block
551    for (int i = 0; i < kvNumberToBeWritten; ++i) {
552      byte[] row = RandomKeyValueUtil.randomOrderedFixedLengthKey(rand, i, 30);
553
554      // Key will be interpreted by KeyValue.KEY_COMPARATOR
555      KeyValue kv = new KeyValue(row, family, qualifier, EnvironmentEdgeManager.currentTime(),
556          RandomKeyValueUtil.randomFixedLengthValue(rand, SMALL_BLOCK_SIZE));
557      writer.append(kv);
558    }
559    writer.close();
560
561    // close hfile.block.index.cacheonwrite
562    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, false);
563
564    // Read the HFile
565    HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
566
567    boolean hasArrayIndexOutOfBoundsException = false;
568    try {
569      // get the mid-key.
570      reader.midKey();
571    } catch (ArrayIndexOutOfBoundsException e) {
572      hasArrayIndexOutOfBoundsException = true;
573    } finally {
574      reader.close();
575    }
576
577    // to check if ArrayIndexOutOfBoundsException occurred
578    assertFalse(hasArrayIndexOutOfBoundsException);
579  }
580
581  /**
582   * Testing block index through the HFile writer/reader APIs. Allows to test
583   * setting index block size through configuration, intermediate-level index
584   * blocks, and caching index blocks on write.
585   *
586   * @throws IOException
587   */
588  @Test
589  public void testHFileWriterAndReader() throws IOException {
590    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
591        "hfile_for_block_index");
592    CacheConfig.instantiateBlockCache(conf);
593    CacheConfig cacheConf = new CacheConfig(conf);
594    BlockCache blockCache = cacheConf.getBlockCache();
595
596    for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) {
597      int indexBlockSize = INDEX_CHUNK_SIZES[testI];
598      int expectedNumLevels = EXPECTED_NUM_LEVELS[testI];
599      LOG.info("Index block size: " + indexBlockSize + ", compression: "
600          + compr);
601      // Evict all blocks that were cached-on-write by the previous invocation.
602      blockCache.evictBlocksByHfileName(hfilePath.getName());
603
604      conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
605      Set<String> keyStrSet = new HashSet<>();
606      byte[][] keys = new byte[NUM_KV][];
607      byte[][] values = new byte[NUM_KV][];
608
609      // Write the HFile
610      {
611        HFileContext meta = new HFileContextBuilder()
612                            .withBlockSize(SMALL_BLOCK_SIZE)
613                            .withCompression(compr)
614                            .build();
615        HFile.Writer writer =
616            HFile.getWriterFactory(conf, cacheConf)
617                .withPath(fs, hfilePath)
618                .withFileContext(meta)
619                .create();
620        Random rand = new Random(19231737);
621        byte[] family = Bytes.toBytes("f");
622        byte[] qualifier = Bytes.toBytes("q");
623        for (int i = 0; i < NUM_KV; ++i) {
624          byte[] row = RandomKeyValueUtil.randomOrderedKey(rand, i);
625
626          // Key will be interpreted by KeyValue.KEY_COMPARATOR
627          KeyValue kv =
628              new KeyValue(row, family, qualifier, EnvironmentEdgeManager.currentTime(),
629                  RandomKeyValueUtil.randomValue(rand));
630          byte[] k = kv.getKey();
631          writer.append(kv);
632          keys[i] = k;
633          values[i] = CellUtil.cloneValue(kv);
634          keyStrSet.add(Bytes.toStringBinary(k));
635          if (i > 0) {
636            assertTrue((PrivateCellUtil.compare(CellComparatorImpl.COMPARATOR, kv, keys[i - 1],
637                0, keys[i - 1].length)) > 0);
638          }
639        }
640
641        writer.close();
642      }
643
644      // Read the HFile
645      HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
646      assertEquals(expectedNumLevels,
647          reader.getTrailer().getNumDataIndexLevels());
648
649      assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey().get()).getKey()));
650      assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey().get()).getKey()));
651      LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1]));
652
653      for (boolean pread : new boolean[] { false, true }) {
654        HFileScanner scanner = reader.getScanner(true, pread);
655        for (int i = 0; i < NUM_KV; ++i) {
656          checkSeekTo(keys, scanner, i);
657          checkKeyValue("i=" + i, keys[i], values[i],
658              ByteBuffer.wrap(((KeyValue) scanner.getKey()).getKey()), scanner.getValue());
659        }
660        assertTrue(scanner.seekTo());
661        for (int i = NUM_KV - 1; i >= 0; --i) {
662          checkSeekTo(keys, scanner, i);
663          checkKeyValue("i=" + i, keys[i], values[i],
664              ByteBuffer.wrap(((KeyValue) scanner.getKey()).getKey()), scanner.getValue());
665        }
666      }
667
668      // Manually compute the mid-key and validate it.
669      HFile.Reader reader2 = reader;
670      HFileBlock.FSReader fsReader = reader2.getUncachedBlockReader();
671
672      HFileBlock.BlockIterator iter = fsReader.blockRange(0,
673          reader.getTrailer().getLoadOnOpenDataOffset());
674      HFileBlock block;
675      List<byte[]> blockKeys = new ArrayList<>();
676      while ((block = iter.nextBlock()) != null) {
677        if (block.getBlockType() != BlockType.LEAF_INDEX)
678          return;
679        ByteBuff b = block.getBufferReadOnly();
680        int n = b.getIntAfterPosition(0);
681        // One int for the number of items, and n + 1 for the secondary index.
682        int entriesOffset = Bytes.SIZEOF_INT * (n + 2);
683
684        // Get all the keys from the leaf index block. S
685        for (int i = 0; i < n; ++i) {
686          int keyRelOffset = b.getIntAfterPosition(Bytes.SIZEOF_INT * (i + 1));
687          int nextKeyRelOffset = b.getIntAfterPosition(Bytes.SIZEOF_INT * (i + 2));
688          int keyLen = nextKeyRelOffset - keyRelOffset;
689          int keyOffset = b.arrayOffset() + entriesOffset + keyRelOffset +
690              HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD;
691          byte[] blockKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset
692              + keyLen);
693          String blockKeyStr = Bytes.toString(blockKey);
694          blockKeys.add(blockKey);
695
696          // If the first key of the block is not among the keys written, we
697          // are not parsing the non-root index block format correctly.
698          assertTrue("Invalid block key from leaf-level block: " + blockKeyStr,
699              keyStrSet.contains(blockKeyStr));
700        }
701      }
702
703      // Validate the mid-key.
704      assertEquals(
705          Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)),
706          reader.midKey());
707
708      assertEquals(UNCOMPRESSED_INDEX_SIZES[testI],
709          reader.getTrailer().getUncompressedDataIndexSize());
710
711      reader.close();
712      reader2.close();
713    }
714  }
715
716  private void checkSeekTo(byte[][] keys, HFileScanner scanner, int i)
717      throws IOException {
718    assertEquals("Failed to seek to key #" + i + " (" + Bytes.toStringBinary(keys[i]) + ")", 0,
719        scanner.seekTo(KeyValueUtil.createKeyValueFromKey(keys[i])));
720  }
721
722  private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr,
723      ByteBuffer buf) {
724    assertEquals(msgPrefix + ": expected " + Bytes.toStringBinary(arr)
725        + ", actual " + Bytes.toStringBinary(buf), 0, Bytes.compareTo(arr, 0,
726        arr.length, buf.array(), buf.arrayOffset(), buf.limit()));
727  }
728
729  /** Check a key/value pair after it was read by the reader */
730  private void checkKeyValue(String msgPrefix, byte[] expectedKey,
731      byte[] expectedValue, ByteBuffer keyRead, ByteBuffer valueRead) {
732    if (!msgPrefix.isEmpty())
733      msgPrefix += ". ";
734
735    assertArrayEqualsBuffer(msgPrefix + "Invalid key", expectedKey, keyRead);
736    assertArrayEqualsBuffer(msgPrefix + "Invalid value", expectedValue,
737        valueRead);
738  }
739
740  @Test
741  public void testIntermediateLevelIndicesWithLargeKeys() throws IOException {
742    testIntermediateLevelIndicesWithLargeKeys(16);
743  }
744
745  @Test
746  public void testIntermediateLevelIndicesWithLargeKeysWithMinNumEntries() throws IOException {
747    // because of the large rowKeys, we will end up with a 50-level block index without sanity check
748    testIntermediateLevelIndicesWithLargeKeys(2);
749  }
750
751  public void testIntermediateLevelIndicesWithLargeKeys(int minNumEntries) throws IOException {
752    Path hfPath = new Path(TEST_UTIL.getDataTestDir(),
753      "testIntermediateLevelIndicesWithLargeKeys.hfile");
754    int maxChunkSize = 1024;
755    FileSystem fs = FileSystem.get(conf);
756    CacheConfig cacheConf = new CacheConfig(conf);
757    conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize);
758    conf.setInt(HFileBlockIndex.MIN_INDEX_NUM_ENTRIES_KEY, minNumEntries);
759    HFileContext context = new HFileContextBuilder().withBlockSize(16).build();
760    HFile.Writer hfw = new HFile.WriterFactory(conf, cacheConf)
761            .withFileContext(context)
762            .withPath(fs, hfPath).create();
763    List<byte[]> keys = new ArrayList<>();
764
765    // This should result in leaf-level indices and a root level index
766    for (int i=0; i < 100; i++) {
767      byte[] rowkey = new byte[maxChunkSize + 1];
768      byte[] b = Bytes.toBytes(i);
769      System.arraycopy(b, 0, rowkey, rowkey.length - b.length, b.length);
770      keys.add(rowkey);
771      hfw.append(CellUtil.createCell(rowkey));
772    }
773    hfw.close();
774
775    HFile.Reader reader = HFile.createReader(fs, hfPath, cacheConf, true, conf);
776    // Scanner doesn't do Cells yet.  Fix.
777    HFileScanner scanner = reader.getScanner(true, true);
778    for (int i = 0; i < keys.size(); ++i) {
779      scanner.seekTo(CellUtil.createCell(keys.get(i)));
780    }
781    reader.close();
782  }
783}
784