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

ndimiduk 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 cbb3840aac7 HBASE-29252: Reduce allocations in RowIndexSeekerV1 (#6902)
cbb3840aac7 is described below

commit cbb3840aac78460ec3e443486e7b3c476717a3a3
Author: Charles Connell <[email protected]>
AuthorDate: Fri Apr 25 05:45:44 2025 -0400

    HBASE-29252: Reduce allocations in RowIndexSeekerV1 (#6902)
    
    Signed-off-by: Nick Dimiduk <[email protected]>
---
 .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java    |  11 ++
 .../hadoop/hbase/io/encoding/RowIndexSeekerV1.java |  40 ++++--
 .../hbase/io/hfile/TestRowIndexV1RoundTrip.java    | 144 +++++++++++++++++++++
 3 files changed, 182 insertions(+), 13 deletions(-)

diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
index a29a98a8c09..8e453fdb985 100644
--- 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
@@ -296,4 +296,15 @@ public class ByteBufferKeyOnlyKeyValue extends 
ByteBufferExtendedCell {
     }
     return ClassSize.align(FIXED_OVERHEAD);
   }
+
+  /**
+   * Completely clears the state of this cell. Useful if you want to reuse 
this object to avoid
+   * allocations.
+   */
+  public void clear() {
+    this.buf = null;
+    this.offset = 0;
+    this.length = 0;
+    this.rowLen = 0;
+  }
 }
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index e283803a143..55096a91fa1 100644
--- 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -84,10 +84,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker 
{
   public Cell getKey() {
     if (current.keyBuffer.hasArray()) {
       return new KeyValue.KeyOnlyKeyValue(current.keyBuffer.array(),
-        current.keyBuffer.arrayOffset() + current.keyBuffer.position(), 
current.keyLength);
+        current.keyBuffer.arrayOffset() + current.keyOffset, 
current.keyLength);
     } else {
       final byte[] key = new byte[current.keyLength];
-      ByteBufferUtils.copyFromBufferToArray(key, current.keyBuffer, 
current.keyBuffer.position(), 0,
+      ByteBufferUtils.copyFromBufferToArray(key, current.keyBuffer, 
current.keyOffset, 0,
         current.keyLength);
       return new KeyValue.KeyOnlyKeyValue(key, 0, current.keyLength);
     }
@@ -254,9 +254,8 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker 
{
     currentBuffer.skip(Bytes.SIZEOF_LONG);
     // key part
     currentBuffer.asSubByteBuffer(currentBuffer.position(), current.keyLength, 
tmpPair);
-    ByteBuffer key = tmpPair.getFirst().duplicate();
-    key.position(tmpPair.getSecond()).limit(tmpPair.getSecond() + 
current.keyLength);
-    current.keyBuffer = key;
+    current.keyBuffer = tmpPair.getFirst();
+    current.keyOffset = tmpPair.getSecond();
     currentBuffer.skip(current.keyLength);
     // value part
     current.valueOffset = currentBuffer.position();
@@ -270,13 +269,12 @@ public class RowIndexSeekerV1 extends 
AbstractEncodedSeeker {
       current.memstoreTS = 0;
     }
     current.nextKvOffset = currentBuffer.position();
-    current.currentKey.setKey(current.keyBuffer, tmpPair.getSecond(), 
current.keyLength);
+    current.currentKey.setKey(current.keyBuffer, current.keyOffset, 
current.keyLength);
   }
 
   protected void decodeTags() {
     current.tagsLength = currentBuffer.getShortAfterPosition(0);
     currentBuffer.skip(Bytes.SIZEOF_SHORT);
-    current.tagsOffset = currentBuffer.position();
     currentBuffer.skip(current.tagsLength);
   }
 
@@ -286,19 +284,35 @@ public class RowIndexSeekerV1 extends 
AbstractEncodedSeeker {
      */
     public final static int KEY_VALUE_LEN_SIZE = 2 * Bytes.SIZEOF_INT;
 
+    // RowIndexSeekerV1 reads one cell at a time from a ByteBuff and uses 
SeekerState's fields to
+    // record the structure of the cell within the ByteBuff.
+
+    // The source of bytes that our cell is backed by
     protected ByteBuff currentBuffer;
+    // Row structure starts at startOffset
     protected int startOffset = -1;
-    protected int valueOffset = -1;
+    // Key starts at keyOffset
+    protected int keyOffset = -1;
+    // Key ends at keyOffset + keyLength
     protected int keyLength;
+    // Value starts at valueOffset
+    protected int valueOffset = -1;
+    // Value ends at valueOffset + valueLength
     protected int valueLength;
+    // Tags start after values and end after tagsLength
     protected int tagsLength = 0;
-    protected int tagsOffset = -1;
 
+    // A ByteBuffer version of currentBuffer that we use to access the key. 
position and limit
+    // are not adjusted so you must use keyOffset and keyLength to know where 
in this ByteBuffer to
+    // read.
     protected ByteBuffer keyBuffer = null;
+    // seqId of the cell being read
     protected long memstoreTS;
+    // Start of the next row structure in currentBuffer
     protected int nextKvOffset;
-    // buffer backed keyonlyKV
-    private ByteBufferKeyOnlyKeyValue currentKey = new 
ByteBufferKeyOnlyKeyValue();
+    // Buffer backed keyonlyKV, cheaply reset and re-used as necessary to 
avoid allocations.
+    // Fed to a comparator in RowIndexSeekerV1#binarySearch().
+    private final ByteBufferKeyOnlyKeyValue currentKey = new 
ByteBufferKeyOnlyKeyValue();
 
     protected boolean isValid() {
       return valueOffset != -1;
@@ -306,7 +320,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker 
{
 
     protected void invalidate() {
       valueOffset = -1;
-      currentKey = new ByteBufferKeyOnlyKeyValue();
+      currentKey.clear();
       currentBuffer = null;
     }
 
@@ -320,13 +334,13 @@ public class RowIndexSeekerV1 extends 
AbstractEncodedSeeker {
         nextState.currentKey.getRowPosition() - Bytes.SIZEOF_SHORT, 
nextState.keyLength);
 
       startOffset = nextState.startOffset;
+      keyOffset = nextState.keyOffset;
       valueOffset = nextState.valueOffset;
       keyLength = nextState.keyLength;
       valueLength = nextState.valueLength;
       nextKvOffset = nextState.nextKvOffset;
       memstoreTS = nextState.memstoreTS;
       currentBuffer = nextState.currentBuffer;
-      tagsOffset = nextState.tagsOffset;
       tagsLength = nextState.tagsLength;
     }
 
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowIndexV1RoundTrip.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowIndexV1RoundTrip.java
new file mode 100644
index 00000000000..2004e20aad6
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowIndexV1RoundTrip.java
@@ -0,0 +1,144 @@
+/*
+ * 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.apache.hadoop.hbase.io.ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparatorImpl;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.SizeCachedNoTagsByteBufferKeyValue;
+import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
+import org.apache.hadoop.hbase.io.ByteBuffAllocator;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.testclassification.IOTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ IOTests.class, MediumTests.class })
+public class TestRowIndexV1RoundTrip {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRowIndexV1RoundTrip.class);
+  private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  private static final DataBlockEncoding DATA_BLOCK_ENCODING = 
DataBlockEncoding.ROW_INDEX_V1;
+  private static final int ENTRY_COUNT = 100;
+
+  private Configuration conf;
+  private FileSystem fs;
+
+  @Before
+  public void setUp() throws IOException {
+    conf = TEST_UTIL.getConfiguration();
+    conf.setLong(MIN_ALLOCATE_SIZE_KEY, 0);
+    fs = FileSystem.get(conf);
+  }
+
+  @Test
+  public void testReadMyWritesOnHeap() throws IOException {
+    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV3");
+    writeDataToHFile(hfilePath, ENTRY_COUNT);
+    readDataFromHFile(hfilePath, ENTRY_COUNT, true);
+  }
+
+  @Test
+  public void testReadMyWritesOnDirectMem() throws IOException {
+    Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV3");
+    writeDataToHFile(hfilePath, ENTRY_COUNT);
+    readDataFromHFile(hfilePath, ENTRY_COUNT, false);
+  }
+
+  private void writeDataToHFile(Path hfilePath, int entryCount) throws 
IOException {
+    HFileContext context =
+      new 
HFileContextBuilder().withBlockSize(1024).withDataBlockEncoding(DATA_BLOCK_ENCODING)
+        .withCellComparator(CellComparatorImpl.COMPARATOR).build();
+    CacheConfig cacheConfig = new CacheConfig(conf);
+    HFile.Writer writer = new HFile.WriterFactory(conf, 
cacheConfig).withPath(fs, hfilePath)
+      .withFileContext(context).create();
+
+    List<KeyValue> keyValues = new ArrayList<>(entryCount);
+
+    writeKeyValues(entryCount, writer, keyValues);
+  }
+
+  private void writeKeyValues(int entryCount, HFile.Writer writer, 
List<KeyValue> keyValues)
+    throws IOException {
+    for (int i = 0; i < entryCount; ++i) {
+      byte[] keyBytes = intToBytes(i);
+
+      byte[] valueBytes = Bytes.toBytes(String.format("value %d", i));
+      KeyValue keyValue = new KeyValue(keyBytes, null, null, valueBytes);
+
+      writer.append(keyValue);
+      keyValues.add(keyValue);
+    }
+    writer.close();
+  }
+
+  private void readDataFromHFile(Path hfilePath, int entryCount, boolean 
onHeap)
+    throws IOException {
+    CacheConfig cacheConfig;
+    if (onHeap) {
+      cacheConfig = new CacheConfig(conf);
+    } else {
+      ByteBuffAllocator allocator = ByteBuffAllocator.create(conf, true);
+      cacheConfig = new CacheConfig(conf, null, null, allocator);
+    }
+    HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConfig, 
false, conf);
+    HFileScanner scanner = reader.getScanner(conf, false, false);
+    scanner.seekTo();
+    int i = 1;
+    while (scanner.next()) {
+      byte[] keyBytes = intToBytes(i);
+      // check row key from getKey() and getCell() separately because they use 
different code paths
+      assertArrayEquals(keyBytes, CellUtil.cloneRow(scanner.getKey()));
+      assertArrayEquals(keyBytes, CellUtil.cloneRow(scanner.getCell()));
+      assertArrayEquals(Bytes.toBytes(String.format("value %d", i)),
+        CellUtil.cloneValue(scanner.getCell()));
+      if (onHeap) {
+        assertTrue(scanner.getCell() instanceof SizeCachedNoTagsKeyValue);
+      } else {
+        assertTrue(scanner.getCell() instanceof 
SizeCachedNoTagsByteBufferKeyValue);
+      }
+      i += 1;
+    }
+    assertEquals(entryCount, i);
+  }
+
+  private byte[] intToBytes(final int i) {
+    ByteBuffer bb = ByteBuffer.allocate(4);
+    bb.putInt(i);
+    return bb.array();
+  }
+}

Reply via email to