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.jupiter.api.Assertions.assertEquals;
021import static org.junit.jupiter.api.Assertions.assertTrue;
022
023import java.io.IOException;
024import java.util.Random;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.fs.FileSystem;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hbase.CellUtil;
029import org.apache.hadoop.hbase.ExtendedCell;
030import org.apache.hadoop.hbase.HBaseTestingUtil;
031import org.apache.hadoop.hbase.HConstants;
032import org.apache.hadoop.hbase.KeyValue;
033import org.apache.hadoop.hbase.PrivateCellUtil;
034import org.apache.hadoop.hbase.fs.HFileSystem;
035import org.apache.hadoop.hbase.regionserver.BloomType;
036import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
037import org.apache.hadoop.hbase.testclassification.IOTests;
038import org.apache.hadoop.hbase.testclassification.MediumTests;
039import org.apache.hadoop.hbase.util.BloomFilterFactory;
040import org.apache.hadoop.hbase.util.BloomFilterUtil;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.junit.jupiter.api.Tag;
043import org.junit.jupiter.api.Test;
044import org.slf4j.Logger;
045import org.slf4j.LoggerFactory;
046
047@Tag(IOTests.TAG)
048@Tag(MediumTests.TAG)
049public class TestSeekBeforeWithInlineBlocks {
050
051  private static final Logger LOG = LoggerFactory.getLogger(TestSeekBeforeWithInlineBlocks.class);
052
053  private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
054
055  private static final int NUM_KV = 10000;
056
057  private static final int DATA_BLOCK_SIZE = 4096;
058  private static final int BLOOM_BLOCK_SIZE = 1024;
059  private static final int[] INDEX_CHUNK_SIZES = { 65536, 4096, 1024 };
060  private static final int[] EXPECTED_NUM_LEVELS = { 1, 2, 3 };
061
062  private static final Random RAND = new Random(192537);
063  private static final byte[] FAM = Bytes.toBytes("family");
064
065  private FileSystem fs;
066  private Configuration conf;
067
068  /**
069   * Scanner.seekBefore() could fail because when seeking to a previous HFile data block, it needs
070   * to know the size of that data block, which it calculates using current data block offset and
071   * the previous data block offset. This fails to work when there are leaf-level index blocks in
072   * the scannable section of the HFile, i.e. starting in HFileV2. This test will try seekBefore()
073   * on a flat (single-level) and multi-level (2,3) HFile and confirm this bug is now fixed. This
074   * bug also happens for inline Bloom blocks for the same reasons.
075   */
076  @Test
077  public void testMultiIndexLevelRandomHFileWithBlooms() throws IOException {
078    conf = TEST_UTIL.getConfiguration();
079    TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
080
081    // Try out different HFile versions to ensure reverse scan works on each version
082    for (int hfileVersion = HFile.MIN_FORMAT_VERSION_WITH_TAGS; hfileVersion
083        <= HFile.MAX_FORMAT_VERSION; hfileVersion++) {
084
085      conf.setInt(HFile.FORMAT_VERSION_KEY, hfileVersion);
086      fs = HFileSystem.get(conf);
087
088      // Try out different bloom types because inline Bloom blocks break seekBefore()
089      for (BloomType bloomType : BloomType.values()) {
090
091        // Test out HFile block indices of various sizes/levels
092        for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; testI++) {
093          int indexBlockSize = INDEX_CHUNK_SIZES[testI];
094          int expectedNumLevels = EXPECTED_NUM_LEVELS[testI];
095
096          LOG.info(String.format("Testing HFileVersion: %s, BloomType: %s, Index Levels: %s",
097            hfileVersion, bloomType, expectedNumLevels));
098
099          conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
100          conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
101          conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
102
103          ExtendedCell[] cells = new ExtendedCell[NUM_KV];
104
105          Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), String.format(
106            "testMultiIndexLevelRandomHFileWithBlooms-%s-%s-%s", hfileVersion, bloomType, testI));
107
108          // Disable caching to prevent it from hiding any bugs in block seeks/reads
109          conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
110          CacheConfig cacheConf = new CacheConfig(conf);
111
112          // Write the HFile
113          {
114            HFileContext meta = new HFileContextBuilder().withBlockSize(DATA_BLOCK_SIZE).build();
115
116            StoreFileWriter storeFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs)
117              .withFilePath(hfilePath).withFileContext(meta).withBloomType(bloomType).build();
118
119            for (int i = 0; i < NUM_KV; i++) {
120              byte[] row = RandomKeyValueUtil.randomOrderedKey(RAND, i);
121              byte[] qual = RandomKeyValueUtil.randomRowOrQualifier(RAND);
122              byte[] value = RandomKeyValueUtil.randomValue(RAND);
123              KeyValue kv = new KeyValue(row, FAM, qual, value);
124
125              storeFileWriter.append(kv);
126              cells[i] = kv;
127            }
128
129            storeFileWriter.close();
130          }
131
132          // Read the HFile
133          HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, true, conf);
134
135          // Sanity check the HFile index level
136          assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels());
137
138          // Check that we can seekBefore in either direction and with both pread
139          // enabled and disabled
140          for (boolean pread : new boolean[] { false, true }) {
141            HFileScanner scanner = reader.getScanner(conf, true, pread);
142            checkNoSeekBefore(cells, scanner, 0);
143            for (int i = 1; i < NUM_KV; i++) {
144              checkSeekBefore(cells, scanner, i);
145              checkCell(cells[i - 1], scanner.getCell());
146            }
147            assertTrue(scanner.seekTo());
148            for (int i = NUM_KV - 1; i >= 1; i--) {
149              checkSeekBefore(cells, scanner, i);
150              checkCell(cells[i - 1], scanner.getCell());
151            }
152            checkNoSeekBefore(cells, scanner, 0);
153            scanner.close();
154          }
155
156          reader.close();
157        }
158      }
159    }
160  }
161
162  private void checkSeekBefore(ExtendedCell[] cells, HFileScanner scanner, int i)
163    throws IOException {
164    assertEquals(true, scanner.seekBefore(cells[i]), "Failed to seek to the key before #" + i + " ("
165      + CellUtil.getCellKeyAsString(cells[i]) + ")");
166  }
167
168  private void checkNoSeekBefore(ExtendedCell[] cells, HFileScanner scanner, int i)
169    throws IOException {
170    assertEquals(false, scanner.seekBefore(cells[i]),
171      "Incorrectly succeeded in seeking to before first key ("
172        + CellUtil.getCellKeyAsString(cells[i]) + ")");
173  }
174
175  /** Check a key/value pair after it was read by the reader */
176  private void checkCell(ExtendedCell expected, ExtendedCell actual) {
177    assertTrue(PrivateCellUtil.equals(expected, actual),
178      String.format("Expected key %s, but was %s", CellUtil.getCellKeyAsString(expected),
179        CellUtil.getCellKeyAsString(actual)));
180  }
181}