You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2023/06/17 13:13:41 UTC

[hbase] branch branch-2.5 updated: HBASE-27940 Midkey metadata in root index block would always be ignored by BlockIndexReader.readMultiLevelIndexRoot (#5298)

This is an automated email from the ASF dual-hosted git repository.

chenglei pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.5 by this push:
     new 62bbc8e3173 HBASE-27940 Midkey metadata in root index block would always be ignored by BlockIndexReader.readMultiLevelIndexRoot (#5298)
62bbc8e3173 is described below

commit 62bbc8e3173540de4db762355af58948f6afd377
Author: chenglei <ch...@apache.org>
AuthorDate: Sat Jun 17 21:13:36 2023 +0800

    HBASE-27940 Midkey metadata in root index block would always be ignored by BlockIndexReader.readMultiLevelIndexRoot (#5298)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../apache/hadoop/hbase/io/hfile/HFileBlock.java   | 103 ++++++++++++---------
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java     |   8 +-
 .../hbase/io/hfile/NoOpIndexBlockEncoder.java      |   8 +-
 .../hadoop/hbase/io/hfile/TestHFileBlockIndex.java |  98 ++++++++++++++++++++
 4 files changed, 165 insertions(+), 52 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 1c7bc73e604..a26424fd7a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -170,7 +170,7 @@ public class HFileBlock implements Cacheable {
    * {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
    * @see Writer#putHeader(byte[], int, int, int, int)
    */
-  private int onDiskDataSizeWithHeader;
+  private final int onDiskDataSizeWithHeader;
   // End of Block Header fields.
 
   /**
@@ -186,13 +186,15 @@ public class HFileBlock implements Cacheable {
    * ByteBuffer-like API across multiple ByteBuffers reading from a cache such as BucketCache. So,
    * we have this ByteBuff type. Unfortunately, it is spread all about HFileBlock. Would be good if
    * could be confined to cache-use only but hard-to-do.
+   * <p>
+   * NOTE: this byteBuff including HFileBlock header and data, but excluding checksum.
    */
-  private ByteBuff buf;
+  private ByteBuff bufWithoutChecksum;
 
   /**
    * Meta data that holds meta information on the hfileblock.
    */
-  private HFileContext fileContext;
+  private final HFileContext fileContext;
 
   /**
    * The offset of this block in the file. Populated by the reader for convenience of access. This
@@ -294,6 +296,8 @@ public class HFileBlock implements Cacheable {
       CacheableDeserializerIdManager.registerDeserializer(BLOCK_DESERIALIZER);
   }
 
+  private final int totalChecksumBytes;
+
   /**
    * Creates a new {@link HFile} block from the given fields. This constructor is used only while
    * writing blocks and caching, and is sitting in a byte buffer and we want to stuff the block into
@@ -330,11 +334,12 @@ public class HFileBlock implements Cacheable {
     this.nextBlockOnDiskSize = nextBlockOnDiskSize;
     this.fileContext = fileContext;
     this.allocator = allocator;
-    this.buf = buf;
+    this.bufWithoutChecksum = buf;
     if (fillHeader) {
       overwriteHeader();
     }
-    this.buf.rewind();
+    this.bufWithoutChecksum.rewind();
+    this.totalChecksumBytes = computeTotalChecksumBytes();
   }
 
   /**
@@ -409,12 +414,12 @@ public class HFileBlock implements Cacheable {
 
   @Override
   public int refCnt() {
-    return buf.refCnt();
+    return bufWithoutChecksum.refCnt();
   }
 
   @Override
   public HFileBlock retain() {
-    buf.retain();
+    bufWithoutChecksum.retain();
     return this;
   }
 
@@ -424,7 +429,7 @@ public class HFileBlock implements Cacheable {
    */
   @Override
   public boolean release() {
-    return buf.release();
+    return bufWithoutChecksum.release();
   }
 
   /**
@@ -439,7 +444,7 @@ public class HFileBlock implements Cacheable {
 
   @Override
   public HFileBlock touch(Object hint) {
-    buf.touch(hint);
+    bufWithoutChecksum.touch(hint);
     return this;
   }
 
@@ -449,7 +454,7 @@ public class HFileBlock implements Cacheable {
       throw new IllegalArgumentException("Querying encoder ID of a block " + "of type other than "
         + BlockType.ENCODED_DATA + ": " + blockType);
     }
-    return buf.getShort(headerSize());
+    return bufWithoutChecksum.getShort(headerSize());
   }
 
   /** Returns the on-disk size of header + data part + checksum. */
@@ -477,15 +482,15 @@ public class HFileBlock implements Cacheable {
    * side-effect.
    */
   private void overwriteHeader() {
-    buf.rewind();
-    blockType.write(buf);
-    buf.putInt(onDiskSizeWithoutHeader);
-    buf.putInt(uncompressedSizeWithoutHeader);
-    buf.putLong(prevBlockOffset);
+    bufWithoutChecksum.rewind();
+    blockType.write(bufWithoutChecksum);
+    bufWithoutChecksum.putInt(onDiskSizeWithoutHeader);
+    bufWithoutChecksum.putInt(uncompressedSizeWithoutHeader);
+    bufWithoutChecksum.putLong(prevBlockOffset);
     if (this.fileContext.isUseHBaseChecksum()) {
-      buf.put(fileContext.getChecksumType().getCode());
-      buf.putInt(fileContext.getBytesPerChecksum());
-      buf.putInt(onDiskDataSizeWithHeader);
+      bufWithoutChecksum.put(fileContext.getChecksumType().getCode());
+      bufWithoutChecksum.putInt(fileContext.getBytesPerChecksum());
+      bufWithoutChecksum.putInt(onDiskDataSizeWithHeader);
     }
   }
 
@@ -505,11 +510,12 @@ public class HFileBlock implements Cacheable {
    * in {@link CompoundBloomFilter} to avoid object creation on every Bloom filter lookup, but has
    * to be used with caution. Buffer holds header, block content, and any follow-on checksums if
    * present.
-   * @return the buffer of this block for read-only operations
+   * @return the buffer of this block for read-only operations,the buffer includes header,but not
+   *         checksum.
    */
   public ByteBuff getBufferReadOnly() {
     // TODO: ByteBuf does not support asReadOnlyBuffer(). Fix.
-    ByteBuff dup = this.buf.duplicate();
+    ByteBuff dup = this.bufWithoutChecksum.duplicate();
     assert dup.position() == 0;
     return dup;
   }
@@ -543,7 +549,7 @@ public class HFileBlock implements Cacheable {
    */
   void sanityCheck() throws IOException {
     // Duplicate so no side-effects
-    ByteBuff dup = this.buf.duplicate().rewind();
+    ByteBuff dup = this.bufWithoutChecksum.duplicate().rewind();
     sanityCheckAssertion(BlockType.read(dup), blockType);
 
     sanityCheckAssertion(dup.getInt(), onDiskSizeWithoutHeader, "onDiskSizeWithoutHeader");
@@ -586,8 +592,8 @@ public class HFileBlock implements Cacheable {
       .append(", prevBlockOffset=").append(prevBlockOffset).append(", isUseHBaseChecksum=")
       .append(fileContext.isUseHBaseChecksum());
     if (fileContext.isUseHBaseChecksum()) {
-      sb.append(", checksumType=").append(ChecksumType.codeToType(this.buf.get(24)))
-        .append(", bytesPerChecksum=").append(this.buf.getInt(24 + 1))
+      sb.append(", checksumType=").append(ChecksumType.codeToType(this.bufWithoutChecksum.get(24)))
+        .append(", bytesPerChecksum=").append(this.bufWithoutChecksum.getInt(24 + 1))
         .append(", onDiskDataSizeWithHeader=").append(onDiskDataSizeWithHeader);
     } else {
       sb.append(", onDiskDataSizeWithHeader=").append(onDiskDataSizeWithHeader).append("(")
@@ -595,9 +601,10 @@ public class HFileBlock implements Cacheable {
         .append(HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM).append(")");
     }
     String dataBegin;
-    if (buf.hasArray()) {
-      dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset() + headerSize(),
-        Math.min(32, buf.limit() - buf.arrayOffset() - headerSize()));
+    if (bufWithoutChecksum.hasArray()) {
+      dataBegin = Bytes.toStringBinary(bufWithoutChecksum.array(),
+        bufWithoutChecksum.arrayOffset() + headerSize(),
+        Math.min(32, bufWithoutChecksum.limit() - bufWithoutChecksum.arrayOffset() - headerSize()));
     } else {
       ByteBuff bufWithoutHeader = getBufferWithoutHeader();
       byte[] dataBeginBytes =
@@ -607,8 +614,8 @@ public class HFileBlock implements Cacheable {
     }
     sb.append(", getOnDiskSizeWithHeader=").append(getOnDiskSizeWithHeader())
       .append(", totalChecksumBytes=").append(totalChecksumBytes()).append(", isUnpacked=")
-      .append(isUnpacked()).append(", buf=[").append(buf).append("]").append(", dataBeginsWith=")
-      .append(dataBegin).append(", fileContext=").append(fileContext)
+      .append(isUnpacked()).append(", buf=[").append(bufWithoutChecksum).append("]")
+      .append(", dataBeginsWith=").append(dataBegin).append(", fileContext=").append(fileContext)
       .append(", nextBlockOnDiskSize=").append(nextBlockOnDiskSize).append("]");
     return sb.toString();
   }
@@ -637,7 +644,7 @@ public class HFileBlock implements Cacheable {
         : reader.getDefaultBlockDecodingContext();
       // Create a duplicated buffer without the header part.
       int headerSize = this.headerSize();
-      ByteBuff dup = this.buf.duplicate();
+      ByteBuff dup = this.bufWithoutChecksum.duplicate();
       dup.position(headerSize);
       dup = dup.slice();
       // Decode the dup into unpacked#buf
@@ -660,7 +667,7 @@ public class HFileBlock implements Cacheable {
     int headerSize = headerSize();
     int capacityNeeded = headerSize + uncompressedSizeWithoutHeader;
 
-    ByteBuff source = buf.duplicate();
+    ByteBuff source = bufWithoutChecksum.duplicate();
     ByteBuff newBuf = allocator.allocate(capacityNeeded);
 
     // Copy header bytes into newBuf.
@@ -679,7 +686,7 @@ public class HFileBlock implements Cacheable {
   public boolean isUnpacked() {
     final int headerSize = headerSize();
     final int expectedCapacity = headerSize + uncompressedSizeWithoutHeader;
-    final int bufCapacity = buf.remaining();
+    final int bufCapacity = bufWithoutChecksum.remaining();
     return bufCapacity == expectedCapacity || bufCapacity == expectedCapacity + headerSize;
   }
 
@@ -695,9 +702,9 @@ public class HFileBlock implements Cacheable {
     return offset;
   }
 
-  /** Returns a byte stream reading the data + checksum of this block */
+  /** Returns a byte stream reading the data(excluding header and checksum) of this block */
   DataInputStream getByteStream() {
-    ByteBuff dup = this.buf.duplicate();
+    ByteBuff dup = this.bufWithoutChecksum.duplicate();
     dup.position(this.headerSize());
     return new DataInputStream(new ByteBuffInputStream(dup));
   }
@@ -706,9 +713,9 @@ public class HFileBlock implements Cacheable {
   public long heapSize() {
     long size = FIXED_OVERHEAD;
     size += fileContext.heapSize();
-    if (buf != null) {
+    if (bufWithoutChecksum != null) {
       // Deep overhead of the byte buffer. Needs to be aligned separately.
-      size += ClassSize.align(buf.capacity() + MULTI_BYTE_BUFFER_HEAP_SIZE);
+      size += ClassSize.align(bufWithoutChecksum.capacity() + MULTI_BYTE_BUFFER_HEAP_SIZE);
     }
     return ClassSize.align(size);
   }
@@ -1832,9 +1839,9 @@ public class HFileBlock implements Cacheable {
   // Cacheable implementation
   @Override
   public int getSerializedLength() {
-    if (buf != null) {
+    if (bufWithoutChecksum != null) {
       // Include extra bytes for block metadata.
-      return this.buf.limit() + BLOCK_METADATA_SPACE;
+      return this.bufWithoutChecksum.limit() + BLOCK_METADATA_SPACE;
     }
     return 0;
   }
@@ -1842,7 +1849,7 @@ public class HFileBlock implements Cacheable {
   // Cacheable implementation
   @Override
   public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
-    this.buf.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
+    this.bufWithoutChecksum.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
     destination = addMetaData(destination, includeNextBlockMetadata);
 
     // Make it ready for reading. flip sets position to zero and limit to current position which
@@ -1888,7 +1895,7 @@ public class HFileBlock implements Cacheable {
     result = result * 31 + onDiskSizeWithoutHeader;
     result = result * 31 + (int) (prevBlockOffset ^ (prevBlockOffset >>> 32));
     result = result * 31 + uncompressedSizeWithoutHeader;
-    result = result * 31 + buf.hashCode();
+    result = result * 31 + bufWithoutChecksum.hashCode();
     return result;
   }
 
@@ -1926,8 +1933,8 @@ public class HFileBlock implements Cacheable {
       return false;
     }
     if (
-      ByteBuff.compareTo(this.buf, 0, this.buf.limit(), castedComparison.buf, 0,
-        castedComparison.buf.limit()) != 0
+      ByteBuff.compareTo(this.bufWithoutChecksum, 0, this.bufWithoutChecksum.limit(),
+        castedComparison.bufWithoutChecksum, 0, castedComparison.bufWithoutChecksum.limit()) != 0
     ) {
       return false;
     }
@@ -1955,10 +1962,17 @@ public class HFileBlock implements Cacheable {
   }
 
   /**
-   * Calculate the number of bytes required to store all the checksums for this block. Each checksum
-   * value is a 4 byte integer.
+   * Return the number of bytes required to store all the checksums for this block. Each checksum
+   * value is a 4 byte integer. <br/>
+   * NOTE: ByteBuff returned by {@link HFileBlock#getBufferWithoutHeader()} and
+   * {@link HFileBlock#getBufferReadOnly} or DataInputStream returned by
+   * {@link HFileBlock#getByteStream()} does not include checksum.
    */
   int totalChecksumBytes() {
+    return totalChecksumBytes;
+  }
+
+  private int computeTotalChecksumBytes() {
     // If the hfile block has minorVersion 0, then there are no checksum
     // data to validate. Similarly, a zero value in this.bytesPerChecksum
     // indicates that cached blocks do not have checksum data because
@@ -2055,7 +2069,8 @@ public class HFileBlock implements Cacheable {
   }
 
   static HFileBlock deepCloneOnHeap(HFileBlock blk) {
-    ByteBuff deepCloned = ByteBuff.wrap(ByteBuffer.wrap(blk.buf.toBytes(0, blk.buf.limit())));
+    ByteBuff deepCloned = ByteBuff
+      .wrap(ByteBuffer.wrap(blk.bufWithoutChecksum.toBytes(0, blk.bufWithoutChecksum.limit())));
     return createBuilder(blk, deepCloned).build();
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index b5a5095c336..12ef197af43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -883,10 +883,10 @@ public class HFileBlockIndex {
      */
     public void readMultiLevelIndexRoot(HFileBlock blk, final int numEntries) throws IOException {
       DataInputStream in = readRootIndex(blk, numEntries);
-      // after reading the root index the checksum bytes have to
-      // be subtracted to know if the mid key exists.
-      int checkSumBytes = blk.totalChecksumBytes();
-      if ((in.available() - checkSumBytes) < MID_KEY_METADATA_SIZE) {
+      // HFileBlock.getByteStream() returns a byte stream for reading the data(excluding checksum)
+      // of root index block, so after reading the root index there is no need to subtract the
+      // checksum bytes.
+      if (in.available() < MID_KEY_METADATA_SIZE) {
         // No mid-key metadata available.
         return;
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpIndexBlockEncoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpIndexBlockEncoder.java
index 9e480247ee9..3115a5153c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpIndexBlockEncoder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpIndexBlockEncoder.java
@@ -204,10 +204,10 @@ public class NoOpIndexBlockEncoder implements HFileIndexBlockEncoder {
 
     private void init(HFileBlock blk, int numEntries) throws IOException {
       DataInputStream in = readRootIndex(blk, numEntries);
-      // after reading the root index the checksum bytes have to
-      // be subtracted to know if the mid key exists.
-      int checkSumBytes = blk.totalChecksumBytes();
-      if ((in.available() - checkSumBytes) < MID_KEY_METADATA_SIZE) {
+      // HFileBlock.getByteStream() returns a byte stream for reading the data(excluding checksum)
+      // of root index block, so after reading the root index there is no need to subtract the
+      // checksum bytes.
+      if (in.available() < MID_KEY_METADATA_SIZE) {
         // No mid-key metadata available.
         return;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index ef5874c0d97..2872065a43e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -42,15 +43,20 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.ByteBuffAllocator;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.encoding.IndexBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
+import org.apache.hadoop.hbase.io.hfile.NoOpIndexBlockEncoder.NoOpEncodedSeeker;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.MultiByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -731,4 +737,96 @@ public class TestHFileBlockIndex {
     }
     reader.close();
   }
+
+  /**
+   * This test is for HBASE-27940, which midkey metadata in root index block would always be ignored
+   * by {@link BlockIndexReader#readMultiLevelIndexRoot}.
+   */
+  @Test
+  public void testMidKeyReadSuccessfullyFromRootIndexBlock() throws IOException {
+    conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, 128);
+    Path hfilePath =
+      new Path(TEST_UTIL.getDataTestDir(), "testMidKeyReadSuccessfullyFromRootIndexBlock");
+    Compression.Algorithm compressAlgo = Compression.Algorithm.NONE;
+    int entryCount = 50000;
+    HFileContext context = new HFileContextBuilder().withBlockSize(4096).withIncludesTags(false)
+      .withDataBlockEncoding(DataBlockEncoding.NONE).withCompression(compressAlgo).build();
+
+    try (HFile.Writer writer = new HFile.WriterFactory(conf, new CacheConfig(conf))
+      .withPath(fs, hfilePath).withFileContext(context).create()) {
+
+      List<KeyValue> keyValues = new ArrayList<>(entryCount);
+      for (int i = 0; i < entryCount; ++i) {
+        byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(RNG, i);
+        // A random-length random value.
+        byte[] valueBytes = RandomKeyValueUtil.randomValue(RNG);
+        KeyValue keyValue =
+          new KeyValue(keyBytes, null, null, HConstants.LATEST_TIMESTAMP, valueBytes);
+        writer.append(keyValue);
+        keyValues.add(keyValue);
+      }
+    }
+
+    try (FSDataInputStream fsdis = fs.open(hfilePath)) {
+      long fileSize = fs.getFileStatus(hfilePath).getLen();
+      FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis, fileSize);
+
+      assertEquals(3, trailer.getMajorVersion());
+      assertEquals(entryCount, trailer.getEntryCount());
+      HFileContext meta = new HFileContextBuilder().withCompression(compressAlgo)
+        .withIncludesMvcc(false).withIncludesTags(false)
+        .withDataBlockEncoding(DataBlockEncoding.NONE).withHBaseCheckSum(true).build();
+      ReaderContext readerContext =
+        new ReaderContextBuilder().withInputStreamWrapper(new FSDataInputStreamWrapper(fsdis))
+          .withFilePath(hfilePath).withFileSystem(fs).withFileSize(fileSize).build();
+      HFileBlock.FSReader blockReader =
+        new HFileBlock.FSReaderImpl(readerContext, meta, ByteBuffAllocator.HEAP, conf);
+
+      MyEncoder encoder = new MyEncoder();
+      HFileBlockIndex.CellBasedKeyBlockIndexReaderV2 dataBlockIndexReader =
+        new HFileBlockIndex.CellBasedKeyBlockIndexReaderV2(trailer.createComparator(),
+          trailer.getNumDataIndexLevels(), encoder);
+
+      HFileBlock.BlockIterator blockIter = blockReader.blockRange(trailer.getLoadOnOpenDataOffset(),
+        fileSize - trailer.getTrailerSize());
+      // Data index. We also read statistics about the block index written after
+      // the root level.
+      dataBlockIndexReader.readMultiLevelIndexRoot(
+        blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), trailer.getDataIndexCount());
+      NoOpEncodedSeeker noOpEncodedSeeker = (NoOpEncodedSeeker) encoder.encoderSeeker;
+      // Assert we have read midkey metadata successfully.
+      assertTrue(noOpEncodedSeeker.midLeafBlockOffset >= 0);
+      assertTrue(noOpEncodedSeeker.midLeafBlockOnDiskSize > 0);
+      assertTrue(noOpEncodedSeeker.midKeyEntry >= 0);
+    }
+  }
+
+  static class MyEncoder implements HFileIndexBlockEncoder {
+
+    EncodedSeeker encoderSeeker;
+
+    @Override
+    public void saveMetadata(Writer writer) throws IOException {
+      NoOpIndexBlockEncoder.INSTANCE.saveMetadata(writer);
+
+    }
+
+    @Override
+    public void encode(BlockIndexChunk blockIndexChunk, boolean rootIndexBlock, DataOutput out)
+      throws IOException {
+      NoOpIndexBlockEncoder.INSTANCE.encode(blockIndexChunk, rootIndexBlock, out);
+    }
+
+    @Override
+    public IndexBlockEncoding getIndexBlockEncoding() {
+      return NoOpIndexBlockEncoder.INSTANCE.getIndexBlockEncoding();
+    }
+
+    @Override
+    public EncodedSeeker createSeeker() {
+      encoderSeeker = NoOpIndexBlockEncoder.INSTANCE.createSeeker();
+      return encoderSeeker;
+    }
+
+  }
 }