HBASE-9807 block encoder unnecessarily copies the key for each reseek
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1535779 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f5881a3116
commit
71e47fb872
|
@ -147,6 +147,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
return this.decodingCtx.getHFileContext().shouldIncludeTags();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
return comparator.compareFlatKey(key, offset, length,
|
||||
current.keyBuffer, 0, current.keyLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setCurrentBuffer(ByteBuffer buffer) {
|
||||
currentBuffer = buffer;
|
||||
|
|
|
@ -177,5 +177,15 @@ public interface DataBlockEncoder {
|
|||
int seekToKeyInBlock(
|
||||
byte[] key, int offset, int length, boolean seekBefore
|
||||
);
|
||||
|
||||
/**
|
||||
* Compare the given key against the current key
|
||||
* @param comparator
|
||||
* @param key
|
||||
* @param offset
|
||||
* @param length
|
||||
* @return -1 is the passed key is smaller than the current key, 0 if equal and 1 if greater
|
||||
*/
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
||||
|
@ -218,4 +219,10 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
throw new RuntimeException("unexpected CellScannerPosition:"+position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
// can't optimize this, make a copy of the key
|
||||
ByteBuffer bb = getKeyDeepCopy();
|
||||
return comparator.compareFlatKey(key, offset, length, bb.array(), bb.arrayOffset(), bb.limit());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
|
||||
|
@ -528,9 +529,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
public int reseekTo(byte[] key, int offset, int length) throws IOException {
|
||||
int compared;
|
||||
if (isSeeked()) {
|
||||
ByteBuffer bb = getKey();
|
||||
compared = reader.getComparator().compareFlatKey(key, offset,
|
||||
length, bb.array(), bb.arrayOffset(), bb.limit());
|
||||
compared = compareKey(reader.getComparator(), key, offset, length);
|
||||
if (compared < 1) {
|
||||
// If the required key is less than or equal to current key, then
|
||||
// don't do anything.
|
||||
|
@ -622,6 +621,16 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
|
||||
return curBlock;
|
||||
}
|
||||
/**
|
||||
* Compare the given key against the current key
|
||||
* @param comparator
|
||||
* @param key
|
||||
* @param offset
|
||||
* @param length
|
||||
* @return -1 is the passed key is smaller than the current key, 0 if equal and 1 if greater
|
||||
*/
|
||||
public abstract int compareKey(KVComparator comparator, byte[] key, int offset,
|
||||
int length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -662,6 +671,12 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
+ KEY_VALUE_LEN_SIZE, currKeyLen).slice();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
return comparator.compareFlatKey(key, offset, length, blockBuffer.array(),
|
||||
blockBuffer.arrayOffset() + blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer getValue() {
|
||||
assertSeeked();
|
||||
|
@ -1067,6 +1082,11 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
return seeker.getKeyDeepCopy();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
return seeker.compareKey(comparator, key, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer getValue() {
|
||||
assertValidSeek();
|
||||
|
|
Loading…
Reference in New Issue