Index: hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java =================================================================== --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java (revision 1533976) +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java (working copy) @@ -148,6 +148,12 @@ } @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; decodeFirst(); Index: hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java =================================================================== --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java (revision 1533976) +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java (working copy) @@ -177,5 +177,15 @@ 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); } } Index: hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java =================================================================== --- hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java (revision 1533976) +++ hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java (working copy) @@ -25,6 +25,7 @@ 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 @@ 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()); + } } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (revision 1533976) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (working copy) @@ -29,6 +29,7 @@ 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 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 @@ 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); } /** @@ -663,6 +672,12 @@ } @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(); return ByteBuffer.wrap( @@ -1068,6 +1083,11 @@ } @Override + public int compareKey(KVComparator comparator, byte[] key, int offset, int length) { + return seeker.compareKey(comparator, key, offset, length); + } + + @Override public ByteBuffer getValue() { assertValidSeek(); return seeker.getValueShallowCopy();