You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2015/10/28 00:29:05 UTC

[06/17] hbase git commit: HBASE-14283 Reverse scan doesn’t work with HFile inline index/bloom blocks

HBASE-14283 Reverse scan doesn’t work with HFile inline index/bloom blocks

Signed-off-by: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5c56e239
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5c56e239
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5c56e239

Branch: refs/heads/hbase-12439
Commit: 5c56e239c3af22e1232681cceaed7bd96480ed92
Parents: efb8295
Author: Ben Lau <be...@yahoo-inc.com>
Authored: Wed Oct 7 16:46:16 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Oct 26 15:12:21 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |   7 +-
 .../hfile/TestSeekBeforeWithInlineBlocks.java   | 189 +++++++++++++++++++
 2 files changed, 194 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5c56e239/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 5af72b6..49b6f5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -859,9 +859,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         reader.returnBlock(seekToBlock);
         // It is important that we compute and pass onDiskSize to the block
         // reader so that it does not have to read the header separately to
-        // figure out the size.
+        // figure out the size.  Currently, we do not have a way to do this
+        // correctly in the general case however.
+        // TODO: See https://issues.apache.org/jira/browse/HBASE-14576
+        int prevBlockSize = -1;
         seekToBlock = reader.readBlock(previousBlockOffset,
-            seekToBlock.getOffset() - previousBlockOffset, cacheBlocks,
+            prevBlockSize, cacheBlocks,
             pread, isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding());
         // TODO shortcut: seek forward in this block to the last key of the
         // block.

http://git-wip-us.apache.org/repos/asf/hbase/blob/5c56e239/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
new file mode 100644
index 0000000..c9af3d3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java
@@ -0,0 +1,189 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.io.hfile;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({IOTests.class, MediumTests.class})
+public class TestSeekBeforeWithInlineBlocks {
+
+  private static final Log LOG = LogFactory.getLog(TestSeekBeforeWithInlineBlocks.class);
+
+  private static final HBaseTestingUtility TEST_UTIL =
+      new HBaseTestingUtility();
+
+  private static final int NUM_KV = 10000;
+
+  private static final int DATA_BLOCK_SIZE = 4096;
+  private static final int BLOOM_BLOCK_SIZE = 1024;
+  private static final int[] INDEX_CHUNK_SIZES = { 65536, 4096, 1024 };
+  private static final int[] EXPECTED_NUM_LEVELS = { 1, 2, 3 };
+
+  private static final Random RAND = new Random(192537);
+  private static final byte[] FAM = Bytes.toBytes("family");
+
+  private FileSystem fs;
+  private Configuration conf;
+
+  /**
+   * Scanner.seekBefore() could fail because when seeking to a previous HFile data block, it needs 
+   * to know the size of that data block, which it calculates using current data block offset and 
+   * the previous data block offset.  This fails to work when there are leaf-level index blocks in 
+   * the scannable section of the HFile, i.e. starting in HFileV2.  This test will try seekBefore() 
+   * on a flat (single-level) and multi-level (2,3) HFile and confirm this bug is now fixed.  This
+   * bug also happens for inline Bloom blocks for the same reasons.
+   */
+  @Test
+  public void testMultiIndexLevelRandomHFileWithBlooms() throws IOException {
+    conf = TEST_UTIL.getConfiguration();
+    
+    // Try out different HFile versions to ensure reverse scan works on each version
+    for (int hfileVersion = HFile.MIN_FORMAT_VERSION_WITH_TAGS; 
+            hfileVersion <= HFile.MAX_FORMAT_VERSION; hfileVersion++) {
+
+      conf.setInt(HFile.FORMAT_VERSION_KEY, hfileVersion);
+      fs = HFileSystem.get(conf);
+      
+      // Try out different bloom types because inline Bloom blocks break seekBefore() 
+      for (BloomType bloomType : BloomType.values()) {
+        
+        // Test out HFile block indices of various sizes/levels
+        for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; testI++) {
+          int indexBlockSize = INDEX_CHUNK_SIZES[testI];
+          int expectedNumLevels = EXPECTED_NUM_LEVELS[testI];
+  
+          LOG.info(String.format("Testing HFileVersion: %s, BloomType: %s, Index Levels: %s", 
+            hfileVersion, bloomType, expectedNumLevels));
+          
+          conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
+          conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
+          
+          Cell[] cells = new Cell[NUM_KV];
+
+          Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
+            String.format("testMultiIndexLevelRandomHFileWithBlooms-%s-%s-%s", 
+              hfileVersion, bloomType, testI));
+          
+          // Disable caching to prevent it from hiding any bugs in block seeks/reads
+          conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
+          CacheConfig cacheConf = new CacheConfig(conf);
+          
+          // Write the HFile
+          {
+            HFileContext meta = new HFileContextBuilder()
+                                .withBlockSize(DATA_BLOCK_SIZE)
+                                .build();
+            
+            StoreFile.Writer storeFileWriter = 
+                new StoreFile.WriterBuilder(conf, cacheConf, fs)
+              .withFilePath(hfilePath)
+              .withFileContext(meta)
+              .withBloomType(bloomType)
+              .build();
+            
+            for (int i = 0; i < NUM_KV; i++) {
+              byte[] row = TestHFileWriterV2.randomOrderedKey(RAND, i);
+              byte[] qual = TestHFileWriterV2.randomRowOrQualifier(RAND);
+              byte[] value = TestHFileWriterV2.randomValue(RAND);
+              KeyValue kv = new KeyValue(row, FAM, qual, value);
+  
+              storeFileWriter.append(kv);
+              cells[i] = kv;
+            }
+  
+            storeFileWriter.close();
+          }
+  
+          // Read the HFile
+          HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, conf);
+          
+          // Sanity check the HFile index level
+          assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels());
+          
+          // Check that we can seekBefore in either direction and with both pread
+          // enabled and disabled
+          for (boolean pread : new boolean[] { false, true }) {
+            HFileScanner scanner = reader.getScanner(true, pread);
+            checkNoSeekBefore(cells, scanner, 0);
+            for (int i = 1; i < NUM_KV; i++) {
+              checkSeekBefore(cells, scanner, i);
+              checkCell(cells[i-1], scanner.getCell());
+            }
+            assertTrue(scanner.seekTo());
+            for (int i = NUM_KV - 1; i >= 1; i--) {
+              checkSeekBefore(cells, scanner, i);
+              checkCell(cells[i-1], scanner.getCell());
+            }
+            checkNoSeekBefore(cells, scanner, 0);
+            scanner.close();
+          }
+  
+          reader.close();
+        }    
+      }
+    }
+  }
+  
+  private void checkSeekBefore(Cell[] cells, HFileScanner scanner, int i)
+      throws IOException {
+    assertEquals("Failed to seek to the key before #" + i + " ("
+        + CellUtil.getCellKeyAsString(cells[i]) + ")", true, 
+        scanner.seekBefore(cells[i]));
+  }
+
+  private void checkNoSeekBefore(Cell[] cells, HFileScanner scanner, int i)
+      throws IOException {
+    assertEquals("Incorrectly succeeded in seeking to before first key ("
+        + CellUtil.getCellKeyAsString(cells[i]) + ")", false, 
+        scanner.seekBefore(cells[i]));
+  }
+
+  /** Check a key/value pair after it was read by the reader */
+  private void checkCell(Cell expected, Cell actual) {
+    assertTrue(String.format("Expected key %s, but was %s", 
+      CellUtil.getCellKeyAsString(expected), CellUtil.getCellKeyAsString(actual)), 
+      CellUtil.equals(expected, actual));
+  }
+}
+