This is an automated email from the ASF dual-hosted git repository. ramkrishna pushed a commit to branch branch-2.2 in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.2 by this push: new 573a6f2 HBASE-25187 Improve SizeCachedKV variants initialization (#2582) 573a6f2 is described below commit 573a6f2b2f2ccd437ce6ae8de7216ff898a2695a Author: ramkrish86 <ram_krish...@hotmail.com> AuthorDate: Wed Nov 11 17:39:39 2020 +0530 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 --- .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java | 14 +++++- ...alue.java => SizeCachedByteBufferKeyValue.java} | 47 +++++++++++++------- .../apache/hadoop/hbase/SizeCachedKeyValue.java | 16 +++++-- ...ava => SizeCachedNoTagsByteBufferKeyValue.java} | 51 ++++++++++++---------- .../hadoop/hbase/SizeCachedNoTagsKeyValue.java | 9 +++- .../hadoop/hbase/io/encoding/RowIndexSeekerV1.java | 22 ++++++---- .../hadoop/hbase/io/hfile/HFileReaderImpl.java | 28 +++++++----- 7 files changed, 123 insertions(+), 64 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 31f71f9..cc7e8d7 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 @@ -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 diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedByteBufferKeyValue.java similarity index 62% copy from hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java copy to hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedByteBufferKeyValue.java index 663f3eb..9f5d9c1 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedByteBufferKeyValue.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,32 +17,39 @@ */ package org.apache.hadoop.hbase; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; /** - * This class is an extension to KeyValue where rowLen and keyLen are cached. - * Parsing the backing byte[] every time to get these values will affect the performance. - * In read path, we tend to read these values many times in Comparator, SQM etc. - * Note: Please do not use these objects in write path as it will increase the heap space usage. - * See https://issues.apache.org/jira/browse/HBASE-13448 + * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in + * off heap/ on heap ByteBuffer */ @InterfaceAudience.Private -@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS") -public class SizeCachedKeyValue extends KeyValue { - // Overhead in this class alone. Parent's overhead will be considered in usage places by calls to - // super. methods - private static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT; +public class SizeCachedByteBufferKeyValue extends ByteBufferKeyValue { + public static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT; private short rowLen; private int keyLen; - public SizeCachedKeyValue(byte[] bytes, int offset, int length, long seqId) { - super(bytes, offset, length); + 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 - rowLen = super.getRowLength(); - keyLen = super.getKeyLength(); + this.rowLen = rowLen; + this.keyLen = keyLen; setSequenceId(seqId); } @@ -72,4 +77,14 @@ public class SizeCachedKeyValue extends KeyValue { public int getSerializedSize() { return this.length; } + + @Override + public boolean equals(Object other) { + return super.equals(other); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java index 663f3eb..5141cfb 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java @@ -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); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsByteBufferKeyValue.java similarity index 53% copy from hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java copy to hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsByteBufferKeyValue.java index 663f3eb..0374169 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsByteBufferKeyValue.java @@ -1,6 +1,4 @@ /** - * Copyright The Apache Software Foundation - * * 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 @@ -19,32 +17,39 @@ */ package org.apache.hadoop.hbase; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; /** - * This class is an extension to KeyValue where rowLen and keyLen are cached. - * Parsing the backing byte[] every time to get these values will affect the performance. - * In read path, we tend to read these values many times in Comparator, SQM etc. - * Note: Please do not use these objects in write path as it will increase the heap space usage. - * See https://issues.apache.org/jira/browse/HBASE-13448 + * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in + * off heap/ on heap ByteBuffer */ @InterfaceAudience.Private -@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS") -public class SizeCachedKeyValue extends KeyValue { - // Overhead in this class alone. Parent's overhead will be considered in usage places by calls to - // super. methods - private static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT; +public class SizeCachedNoTagsByteBufferKeyValue extends NoTagsByteBufferKeyValue { + public static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT; private short rowLen; private int keyLen; - public SizeCachedKeyValue(byte[] bytes, int offset, int length, long seqId) { - super(bytes, offset, length); + 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 - rowLen = super.getRowLength(); - keyLen = super.getKeyLength(); + this.rowLen = rowLen; + this.keyLen = keyLen; setSequenceId(seqId); } @@ -63,13 +68,13 @@ public class SizeCachedKeyValue extends KeyValue { 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; + public boolean equals(Object other) { + return super.equals(other); + } + + @Override + public int hashCode() { + return super.hashCode(); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsKeyValue.java index 88b6177..85bdb52 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SizeCachedNoTagsKeyValue.java @@ -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 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 14d847c..ae4a543 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 @@ -20,15 +20,15 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hbase.ByteBufferExtendedCell; 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; @@ -374,26 +374,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); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index d1b3a89..dfe57e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -34,11 +34,11 @@ 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.NoTagsByteBufferKeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.ByteBufferKeyValue; +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.trace.TraceUtil; import org.apache.yetus.audience.InterfaceAudience; @@ -497,6 +497,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { protected AtomicInteger blockFetches = new AtomicInteger(0); 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 @@ -627,6 +628,7 @@ public 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()) { @@ -735,8 +737,9 @@ public 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; @@ -978,23 +981,28 @@ public 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); } } } @@ -1260,7 +1268,7 @@ public 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); }