HBASE-25187 Improve SizeCachedKV variants initialization (#2582)

* HBASE-25187 Improve SizeCachedKV variants initialization

* HBASE-25187 Improve SizeCachedKV variants initialization

* The BBKeyValue also can be optimized

* Change for SizeCachedKeyValue

* Addressing revew comments

* Fixing checkstyle and spot bugs comments

* Spot bug fix for hashCode

* Minor updates make the rowLen as short and some consturctor formatting

* Change two more places where there was a cast
This commit is contained in:
ramkrish86 2020-11-11 17:39:39 +05:30 committed by GitHub
parent f0c430aed2
commit 57d9cae480
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 234 additions and 25 deletions

View File

@ -61,10 +61,22 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
* @param length
*/
public void setKey(ByteBuffer key, int offset, int length) {
setKey(key, offset, length, ByteBufferUtils.toShort(key, offset));
}
/**
* A setter that helps to avoid object creation every time and whenever
* there is a need to create new OffheapKeyOnlyKeyValue.
* @param key - the key part of the cell
* @param offset - offset of the cell
* @param length - length of the cell
* @param rowLen - the rowlen part of the cell
*/
public void setKey(ByteBuffer key, int offset, int length, short rowLen) {
this.buf = key;
this.offset = offset;
this.length = length;
this.rowLen = ByteBufferUtils.toShort(this.buf, this.offset);
this.rowLen = rowLen;
}
@Override

View File

@ -0,0 +1,90 @@
/**
* 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;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
* off heap/ on heap ByteBuffer
*/
@InterfaceAudience.Private
public class SizeCachedByteBufferKeyValue extends ByteBufferKeyValue {
public static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT;
private short rowLen;
private int keyLen;
public SizeCachedByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId,
int keyLen) {
super(buf, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
this.rowLen = super.getRowLength();
this.keyLen = keyLen;
setSequenceId(seqId);
}
public SizeCachedByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId,
int keyLen, short rowLen) {
super(buf, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
this.rowLen = rowLen;
this.keyLen = keyLen;
setSequenceId(seqId);
}
@Override
public short getRowLength() {
return rowLen;
}
@Override
public int getKeyLength() {
return this.keyLen;
}
@Override
public long heapSize() {
return super.heapSize() + FIXED_OVERHEAD;
}
/**
* Override by just returning the length for saving cost of method dispatching. If not, it will
* call {@link ExtendedCell#getSerializedSize()} firstly, then forward to
* {@link SizeCachedKeyValue#getSerializedSize(boolean)}. (See HBASE-21657)
*/
@Override
public int getSerializedSize() {
return this.length;
}
@Override
public boolean equals(Object other) {
return super.equals(other);
}
@Override
public int hashCode() {
return super.hashCode();
}
}

View File

@ -39,12 +39,22 @@ public class SizeCachedKeyValue extends KeyValue {
private short rowLen;
private int keyLen;
public SizeCachedKeyValue(byte[] bytes, int offset, int length, long seqId) {
public SizeCachedKeyValue(byte[] bytes, int offset, int length, long seqId, int keyLen) {
super(bytes, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
rowLen = super.getRowLength();
keyLen = super.getKeyLength();
this.rowLen = super.getRowLength();
this.keyLen = keyLen;
setSequenceId(seqId);
}
public SizeCachedKeyValue(byte[] bytes, int offset, int length, long seqId, int keyLen,
short rowLen) {
super(bytes, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
this.rowLen = rowLen;
this.keyLen = keyLen;
setSequenceId(seqId);
}

View File

@ -0,0 +1,80 @@
/**
* 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;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
* off heap/ on heap ByteBuffer
*/
@InterfaceAudience.Private
public class SizeCachedNoTagsByteBufferKeyValue extends NoTagsByteBufferKeyValue {
public static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT;
private short rowLen;
private int keyLen;
public SizeCachedNoTagsByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId,
int keyLen) {
super(buf, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
this.rowLen = super.getRowLength();
this.keyLen = keyLen;
setSequenceId(seqId);
}
public SizeCachedNoTagsByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId,
int keyLen, short rowLen) {
super(buf, offset, length);
// We will read all these cached values at least once. Initialize now itself so that we can
// avoid uninitialized checks with every time call
this.rowLen = rowLen;
this.keyLen = keyLen;
setSequenceId(seqId);
}
@Override
public short getRowLength() {
return rowLen;
}
@Override
public int getKeyLength() {
return this.keyLen;
}
@Override
public long heapSize() {
return super.heapSize() + FIXED_OVERHEAD;
}
@Override
public boolean equals(Object other) {
return super.equals(other);
}
@Override
public int hashCode() {
return super.hashCode();
}
}

View File

@ -32,8 +32,13 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class SizeCachedNoTagsKeyValue extends SizeCachedKeyValue {
public SizeCachedNoTagsKeyValue(byte[] bytes, int offset, int length, long seqId) {
super(bytes, offset, length, seqId);
public SizeCachedNoTagsKeyValue(byte[] bytes, int offset, int length, long seqId, int keyLen) {
super(bytes, offset, length, seqId, keyLen);
}
public SizeCachedNoTagsKeyValue(byte[] bytes, int offset, int length, long seqId, int keyLen,
short rowLen) {
super(bytes, offset, length, seqId, keyLen, rowLen);
}
@Override

View File

@ -18,15 +18,15 @@ package org.apache.hadoop.hbase.io.encoding;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.SizeCachedByteBufferKeyValue;
import org.apache.hadoop.hbase.SizeCachedKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsByteBufferKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
import org.apache.hadoop.hbase.io.encoding.AbstractDataBlockEncoder.AbstractEncodedSeeker;
import org.apache.hadoop.hbase.nio.ByteBuff;
@ -359,26 +359,30 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
// TODO : reduce the varieties of KV here. Check if based on a boolean
// we can handle the 'no tags' case.
if (tagsLength > 0) {
// TODO : getRow len here.
ret = new SizeCachedKeyValue(currentBuffer.array(),
currentBuffer.arrayOffset() + startOffset, cellBufSize, seqId);
currentBuffer.arrayOffset() + startOffset, cellBufSize, seqId, keyLength);
} else {
ret = new SizeCachedNoTagsKeyValue(currentBuffer.array(),
currentBuffer.arrayOffset() + startOffset, cellBufSize, seqId);
currentBuffer.arrayOffset() + startOffset, cellBufSize, seqId, keyLength);
}
} else {
currentBuffer.asSubByteBuffer(startOffset, cellBufSize, tmpPair);
ByteBuffer buf = tmpPair.getFirst();
if (buf.isDirect()) {
ret =
tagsLength > 0 ? new ByteBufferKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId)
: new NoTagsByteBufferKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId);
// TODO : getRow len here.
ret = tagsLength > 0
? new SizeCachedByteBufferKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId,
keyLength)
: new SizeCachedNoTagsByteBufferKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId,
keyLength);
} else {
if (tagsLength > 0) {
ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset()
+ tmpPair.getSecond(), cellBufSize, seqId);
+ tmpPair.getSecond(), cellBufSize, seqId, keyLength);
} else {
ret = new SizeCachedNoTagsKeyValue(buf.array(), buf.arrayOffset()
+ tmpPair.getSecond(), cellBufSize, seqId);
+ tmpPair.getSecond(), cellBufSize, seqId, keyLength);
}
}
}

View File

@ -26,15 +26,15 @@ import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.SizeCachedByteBufferKeyValue;
import org.apache.hadoop.hbase.SizeCachedKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsByteBufferKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
@ -322,6 +322,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
private long currMemstoreTS;
protected final HFile.Reader reader;
private int currTagsLen;
private short rowLen;
// buffer backed keyonlyKV
private ByteBufferKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferKeyOnlyKeyValue();
// A pair for reusing in blockSeek() so that we don't garbage lot of objects
@ -446,6 +447,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
this.currKeyLen = (int)(ll >> Integer.SIZE);
this.currValueLen = (int)(Bytes.MASK_FOR_LOWER_INT_IN_LONG ^ ll);
checkKeyValueLen();
this.rowLen = blockBuffer.getShortAfterPosition(Bytes.SIZEOF_LONG);
// Move position past the key and value lengths and then beyond the key and value
int p = (Bytes.SIZEOF_LONG + currKeyLen + currValueLen);
if (reader.getFileContext().isIncludesTags()) {
@ -554,8 +556,9 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
+ " path=" + reader.getPath());
}
offsetFromPos += Bytes.SIZEOF_LONG;
this.rowLen = blockBuffer.getShortAfterPosition(offsetFromPos);
blockBuffer.asSubByteBuffer(blockBuffer.position() + offsetFromPos, klen, pair);
bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), klen);
bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), klen, rowLen);
int comp =
PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, bufBackedKeyOnlyKv);
offsetFromPos += klen + vlen;
@ -790,23 +793,28 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
// we can handle the 'no tags' case.
if (currTagsLen > 0) {
ret = new SizeCachedKeyValue(blockBuffer.array(),
blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId);
blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId, currKeyLen,
rowLen);
} else {
ret = new SizeCachedNoTagsKeyValue(blockBuffer.array(),
blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId);
blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId, currKeyLen,
rowLen);
}
} else {
ByteBuffer buf = blockBuffer.asSubByteBuffer(cellBufSize);
if (buf.isDirect()) {
ret = currTagsLen > 0 ? new ByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId)
: new NoTagsByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId);
ret = currTagsLen > 0
? new SizeCachedByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId,
currKeyLen, rowLen)
: new SizeCachedNoTagsByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId,
currKeyLen, rowLen);
} else {
if (currTagsLen > 0) {
ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset() + buf.position(),
cellBufSize, seqId);
cellBufSize, seqId, currKeyLen, rowLen);
} else {
ret = new SizeCachedNoTagsKeyValue(buf.array(), buf.arrayOffset() + buf.position(),
cellBufSize, seqId);
cellBufSize, seqId, currKeyLen, rowLen);
}
}
}
@ -1060,7 +1068,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
public int compareKey(CellComparator comparator, Cell key) {
blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, pair);
this.bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), currKeyLen);
this.bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), currKeyLen, rowLen);
return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, this.bufBackedKeyOnlyKv);
}