HBASE-13448 New Cell implementation with cached component offsets/lengths.
This commit is contained in:
parent
1d7c68c086
commit
a4bd2b784e
|
@ -147,6 +147,8 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
Bytes.SIZEOF_INT /*keylength*/ +
|
||||
Bytes.SIZEOF_INT /*valuelength*/;
|
||||
|
||||
public static final int ROW_KEY_OFFSET = ROW_OFFSET + ROW_LENGTH_SIZE;
|
||||
|
||||
// Size of the length ints in a KeyValue datastructure.
|
||||
public static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET;
|
||||
|
||||
|
@ -1328,7 +1330,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
*/
|
||||
@Override
|
||||
public int getRowOffset() {
|
||||
return getKeyOffset() + Bytes.SIZEOF_SHORT;
|
||||
return this.offset + ROW_KEY_OFFSET;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1359,7 +1361,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
* @return Family offset
|
||||
*/
|
||||
private int getFamilyOffset(int rlength) {
|
||||
return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
|
||||
return this.offset + ROW_KEY_OFFSET + rlength + Bytes.SIZEOF_BYTE;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* 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
|
||||
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SizeCachedKeyValue extends KeyValue {
|
||||
|
||||
private static final int HEAP_SIZE_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT;
|
||||
|
||||
private short rowLen;
|
||||
private int keyLen;
|
||||
|
||||
public SizeCachedKeyValue(byte[] bytes, int offset, int length) {
|
||||
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();
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getRowLength() {
|
||||
return rowLen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getKeyLength() {
|
||||
return this.keyLen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
return super.heapSize() + HEAP_SIZE_OVERHEAD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSizeWithoutTags() {
|
||||
return super.heapSizeWithoutTags() + HEAP_SIZE_OVERHEAD;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/**
|
||||
* 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
|
||||
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* This class is an extension to ContentSizeCachedKeyValue where there are no tags in Cell.
|
||||
* 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
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SizeCachedNoTagsKeyValue extends SizeCachedKeyValue {
|
||||
|
||||
public SizeCachedNoTagsKeyValue(byte[] bytes, int offset, int length) {
|
||||
super(bytes, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTagsLength() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSizeWithoutTags() {
|
||||
return super.heapSize();
|
||||
}
|
||||
}
|
|
@ -35,10 +35,11 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.SizeCachedKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
|
@ -827,22 +828,19 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
if (!isSeeked())
|
||||
return null;
|
||||
|
||||
if(currTagsLen > 0) {
|
||||
KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
|
||||
KeyValue ret;
|
||||
if (currTagsLen > 0) {
|
||||
ret = new SizeCachedKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
|
||||
+ blockBuffer.position(), getCellBufSize());
|
||||
if (this.reader.shouldIncludeMemstoreTS()) {
|
||||
ret.setSequenceId(currMemstoreTS);
|
||||
}
|
||||
return ret;
|
||||
} else {
|
||||
NoTagsKeyValue ret = new NoTagsKeyValue(blockBuffer.array(),
|
||||
blockBuffer.arrayOffset() + blockBuffer.position(), getCellBufSize());
|
||||
ret = new SizeCachedNoTagsKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
|
||||
+ blockBuffer.position(), getCellBufSize());
|
||||
}
|
||||
if (this.reader.shouldIncludeMemstoreTS()) {
|
||||
ret.setSequenceId(currMemstoreTS);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer getKey() {
|
||||
|
|
Loading…
Reference in New Issue