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