-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-29252: Reduce allocations in RowIndexSeekerV1 #6902
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
ndimiduk
merged 2 commits into
apache:master
from
HubSpot:HBASE-29252/row-index-v1-allocations
Apr 25, 2025
+182
−13
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -85,10 +85,10 @@ public void setCurrentBuffer(ByteBuff buffer) { | |
| public ExtendedCell 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); | ||
| } | ||
|
|
@@ -255,9 +255,8 @@ protected void decodeNext() { | |
| 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(); | ||
|
|
@@ -271,13 +270,12 @@ protected void decodeNext() { | |
| 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); | ||
| } | ||
|
|
||
|
|
@@ -287,27 +285,43 @@ private class SeekerState { | |
| */ | ||
| 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; | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I noticed that |
||
|
|
||
| // 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; | ||
| } | ||
|
|
||
| protected void invalidate() { | ||
| valueOffset = -1; | ||
charlesconnell marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| currentKey = new ByteBufferKeyOnlyKeyValue(); | ||
| currentKey.clear(); | ||
| currentBuffer = null; | ||
| } | ||
|
|
||
|
|
@@ -321,13 +335,13 @@ protected void copyFromNext(SeekerState nextState) { | |
| 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; | ||
| } | ||
|
|
||
|
|
||
144 changes: 144 additions & 0 deletions
144
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestRowIndexV1RoundTrip.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.HBaseTestingUtil; | ||
| 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 HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); | ||
| 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(); | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Better add some comments to explain what do these offsets and lengths mean?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I won't request further changes, but for future reference, these comments would ideally be javadoc and then this additional information would be more readily onhand for folks pursuing the code in an IDE environment.