Skip to content

Commit 972acd1

Browse files
charlesconnellndimiduk
authored andcommitted
HBASE-29252: Reduce allocations in RowIndexSeekerV1 (apache#6902)
Signed-off-by: Nick Dimiduk <[email protected]>
1 parent 47b13dd commit 972acd1

File tree

3 files changed

+182
-13
lines changed

3 files changed

+182
-13
lines changed

hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -296,4 +296,15 @@ public long heapSize() {
296296
}
297297
return ClassSize.align(FIXED_OVERHEAD);
298298
}
299+
300+
/**
301+
* Completely clears the state of this cell. Useful if you want to reuse this object to avoid
302+
* allocations.
303+
*/
304+
public void clear() {
305+
this.buf = null;
306+
this.offset = 0;
307+
this.length = 0;
308+
this.rowLen = 0;
309+
}
299310
}

hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java

Lines changed: 27 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -84,10 +84,10 @@ public void setCurrentBuffer(ByteBuff buffer) {
8484
public Cell getKey() {
8585
if (current.keyBuffer.hasArray()) {
8686
return new KeyValue.KeyOnlyKeyValue(current.keyBuffer.array(),
87-
current.keyBuffer.arrayOffset() + current.keyBuffer.position(), current.keyLength);
87+
current.keyBuffer.arrayOffset() + current.keyOffset, current.keyLength);
8888
} else {
8989
final byte[] key = new byte[current.keyLength];
90-
ByteBufferUtils.copyFromBufferToArray(key, current.keyBuffer, current.keyBuffer.position(), 0,
90+
ByteBufferUtils.copyFromBufferToArray(key, current.keyBuffer, current.keyOffset, 0,
9191
current.keyLength);
9292
return new KeyValue.KeyOnlyKeyValue(key, 0, current.keyLength);
9393
}
@@ -254,9 +254,8 @@ protected void decodeNext() {
254254
currentBuffer.skip(Bytes.SIZEOF_LONG);
255255
// key part
256256
currentBuffer.asSubByteBuffer(currentBuffer.position(), current.keyLength, tmpPair);
257-
ByteBuffer key = tmpPair.getFirst().duplicate();
258-
key.position(tmpPair.getSecond()).limit(tmpPair.getSecond() + current.keyLength);
259-
current.keyBuffer = key;
257+
current.keyBuffer = tmpPair.getFirst();
258+
current.keyOffset = tmpPair.getSecond();
260259
currentBuffer.skip(current.keyLength);
261260
// value part
262261
current.valueOffset = currentBuffer.position();
@@ -270,13 +269,12 @@ protected void decodeNext() {
270269
current.memstoreTS = 0;
271270
}
272271
current.nextKvOffset = currentBuffer.position();
273-
current.currentKey.setKey(current.keyBuffer, tmpPair.getSecond(), current.keyLength);
272+
current.currentKey.setKey(current.keyBuffer, current.keyOffset, current.keyLength);
274273
}
275274

276275
protected void decodeTags() {
277276
current.tagsLength = currentBuffer.getShortAfterPosition(0);
278277
currentBuffer.skip(Bytes.SIZEOF_SHORT);
279-
current.tagsOffset = currentBuffer.position();
280278
currentBuffer.skip(current.tagsLength);
281279
}
282280

@@ -286,27 +284,43 @@ private class SeekerState {
286284
*/
287285
public final static int KEY_VALUE_LEN_SIZE = 2 * Bytes.SIZEOF_INT;
288286

287+
// RowIndexSeekerV1 reads one cell at a time from a ByteBuff and uses SeekerState's fields to
288+
// record the structure of the cell within the ByteBuff.
289+
290+
// The source of bytes that our cell is backed by
289291
protected ByteBuff currentBuffer;
292+
// Row structure starts at startOffset
290293
protected int startOffset = -1;
291-
protected int valueOffset = -1;
294+
// Key starts at keyOffset
295+
protected int keyOffset = -1;
296+
// Key ends at keyOffset + keyLength
292297
protected int keyLength;
298+
// Value starts at valueOffset
299+
protected int valueOffset = -1;
300+
// Value ends at valueOffset + valueLength
293301
protected int valueLength;
302+
// Tags start after values and end after tagsLength
294303
protected int tagsLength = 0;
295-
protected int tagsOffset = -1;
296304

305+
// A ByteBuffer version of currentBuffer that we use to access the key. position and limit
306+
// are not adjusted so you must use keyOffset and keyLength to know where in this ByteBuffer to
307+
// read.
297308
protected ByteBuffer keyBuffer = null;
309+
// seqId of the cell being read
298310
protected long memstoreTS;
311+
// Start of the next row structure in currentBuffer
299312
protected int nextKvOffset;
300-
// buffer backed keyonlyKV
301-
private ByteBufferKeyOnlyKeyValue currentKey = new ByteBufferKeyOnlyKeyValue();
313+
// Buffer backed keyonlyKV, cheaply reset and re-used as necessary to avoid allocations.
314+
// Fed to a comparator in RowIndexSeekerV1#binarySearch().
315+
private final ByteBufferKeyOnlyKeyValue currentKey = new ByteBufferKeyOnlyKeyValue();
302316

303317
protected boolean isValid() {
304318
return valueOffset != -1;
305319
}
306320

307321
protected void invalidate() {
308322
valueOffset = -1;
309-
currentKey = new ByteBufferKeyOnlyKeyValue();
323+
currentKey.clear();
310324
currentBuffer = null;
311325
}
312326

@@ -320,13 +334,13 @@ protected void copyFromNext(SeekerState nextState) {
320334
nextState.currentKey.getRowPosition() - Bytes.SIZEOF_SHORT, nextState.keyLength);
321335

322336
startOffset = nextState.startOffset;
337+
keyOffset = nextState.keyOffset;
323338
valueOffset = nextState.valueOffset;
324339
keyLength = nextState.keyLength;
325340
valueLength = nextState.valueLength;
326341
nextKvOffset = nextState.nextKvOffset;
327342
memstoreTS = nextState.memstoreTS;
328343
currentBuffer = nextState.currentBuffer;
329-
tagsOffset = nextState.tagsOffset;
330344
tagsLength = nextState.tagsLength;
331345
}
332346

Lines changed: 144 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,144 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.io.hfile;
19+
20+
import static org.apache.hadoop.hbase.io.ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY;
21+
import static org.junit.Assert.assertArrayEquals;
22+
import static org.junit.Assert.assertEquals;
23+
import static org.junit.Assert.assertTrue;
24+
25+
import java.io.IOException;
26+
import java.nio.ByteBuffer;
27+
import java.util.ArrayList;
28+
import java.util.List;
29+
import org.apache.hadoop.conf.Configuration;
30+
import org.apache.hadoop.fs.FileSystem;
31+
import org.apache.hadoop.fs.Path;
32+
import org.apache.hadoop.hbase.CellComparatorImpl;
33+
import org.apache.hadoop.hbase.CellUtil;
34+
import org.apache.hadoop.hbase.HBaseClassTestRule;
35+
import org.apache.hadoop.hbase.HBaseTestingUtility;
36+
import org.apache.hadoop.hbase.KeyValue;
37+
import org.apache.hadoop.hbase.SizeCachedNoTagsByteBufferKeyValue;
38+
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
39+
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
40+
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
41+
import org.apache.hadoop.hbase.testclassification.IOTests;
42+
import org.apache.hadoop.hbase.testclassification.MediumTests;
43+
import org.apache.hadoop.hbase.util.Bytes;
44+
import org.junit.Before;
45+
import org.junit.ClassRule;
46+
import org.junit.Test;
47+
import org.junit.experimental.categories.Category;
48+
49+
@Category({ IOTests.class, MediumTests.class })
50+
public class TestRowIndexV1RoundTrip {
51+
@ClassRule
52+
public static final HBaseClassTestRule CLASS_RULE =
53+
HBaseClassTestRule.forClass(TestRowIndexV1RoundTrip.class);
54+
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
55+
private static final DataBlockEncoding DATA_BLOCK_ENCODING = DataBlockEncoding.ROW_INDEX_V1;
56+
private static final int ENTRY_COUNT = 100;
57+
58+
private Configuration conf;
59+
private FileSystem fs;
60+
61+
@Before
62+
public void setUp() throws IOException {
63+
conf = TEST_UTIL.getConfiguration();
64+
conf.setLong(MIN_ALLOCATE_SIZE_KEY, 0);
65+
fs = FileSystem.get(conf);
66+
}
67+
68+
@Test
69+
public void testReadMyWritesOnHeap() throws IOException {
70+
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV3");
71+
writeDataToHFile(hfilePath, ENTRY_COUNT);
72+
readDataFromHFile(hfilePath, ENTRY_COUNT, true);
73+
}
74+
75+
@Test
76+
public void testReadMyWritesOnDirectMem() throws IOException {
77+
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(), "testHFileFormatV3");
78+
writeDataToHFile(hfilePath, ENTRY_COUNT);
79+
readDataFromHFile(hfilePath, ENTRY_COUNT, false);
80+
}
81+
82+
private void writeDataToHFile(Path hfilePath, int entryCount) throws IOException {
83+
HFileContext context =
84+
new HFileContextBuilder().withBlockSize(1024).withDataBlockEncoding(DATA_BLOCK_ENCODING)
85+
.withCellComparator(CellComparatorImpl.COMPARATOR).build();
86+
CacheConfig cacheConfig = new CacheConfig(conf);
87+
HFile.Writer writer = new HFile.WriterFactory(conf, cacheConfig).withPath(fs, hfilePath)
88+
.withFileContext(context).create();
89+
90+
List<KeyValue> keyValues = new ArrayList<>(entryCount);
91+
92+
writeKeyValues(entryCount, writer, keyValues);
93+
}
94+
95+
private void writeKeyValues(int entryCount, HFile.Writer writer, List<KeyValue> keyValues)
96+
throws IOException {
97+
for (int i = 0; i < entryCount; ++i) {
98+
byte[] keyBytes = intToBytes(i);
99+
100+
byte[] valueBytes = Bytes.toBytes(String.format("value %d", i));
101+
KeyValue keyValue = new KeyValue(keyBytes, null, null, valueBytes);
102+
103+
writer.append(keyValue);
104+
keyValues.add(keyValue);
105+
}
106+
writer.close();
107+
}
108+
109+
private void readDataFromHFile(Path hfilePath, int entryCount, boolean onHeap)
110+
throws IOException {
111+
CacheConfig cacheConfig;
112+
if (onHeap) {
113+
cacheConfig = new CacheConfig(conf);
114+
} else {
115+
ByteBuffAllocator allocator = ByteBuffAllocator.create(conf, true);
116+
cacheConfig = new CacheConfig(conf, null, null, allocator);
117+
}
118+
HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConfig, false, conf);
119+
HFileScanner scanner = reader.getScanner(conf, false, false);
120+
scanner.seekTo();
121+
int i = 1;
122+
while (scanner.next()) {
123+
byte[] keyBytes = intToBytes(i);
124+
// check row key from getKey() and getCell() separately because they use different code paths
125+
assertArrayEquals(keyBytes, CellUtil.cloneRow(scanner.getKey()));
126+
assertArrayEquals(keyBytes, CellUtil.cloneRow(scanner.getCell()));
127+
assertArrayEquals(Bytes.toBytes(String.format("value %d", i)),
128+
CellUtil.cloneValue(scanner.getCell()));
129+
if (onHeap) {
130+
assertTrue(scanner.getCell() instanceof SizeCachedNoTagsKeyValue);
131+
} else {
132+
assertTrue(scanner.getCell() instanceof SizeCachedNoTagsByteBufferKeyValue);
133+
}
134+
i += 1;
135+
}
136+
assertEquals(entryCount, i);
137+
}
138+
139+
private byte[] intToBytes(final int i) {
140+
ByteBuffer bb = ByteBuffer.allocate(4);
141+
bb.putInt(i);
142+
return bb.array();
143+
}
144+
}

0 commit comments

Comments
 (0)