diff --git 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 index d79aff2..9178c37 100644 --- 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 @@ -77,6 +77,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { protected static class SeekerState { protected int valueOffset = -1; protected int keyLength; + protected int keyOffset = 0; protected int valueLength; protected int lastCommonPrefix; protected int tagsLength = 0; @@ -85,7 +86,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { protected boolean uncompressTags = true; /** We need to store a copy of the key. */ - protected byte[] keyBuffer = new byte[INITIAL_KEY_BUFFER_SIZE]; + // WE will create a keyBuffer equivalent to currentBuffer and keep it valid till the block is read + // So this would help in just moving the offsets and lengths and would avoid copying too + protected ByteBuffer keyBuffer; protected byte[] tagsBuffer = new byte[INITIAL_KEY_BUFFER_SIZE]; protected long memstoreTS; @@ -99,21 +102,23 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { valueOffset = -1; tagsCompressedLength = 0; uncompressTags = true; + keyOffset = 0; } protected void ensureSpaceForKey() { - if (keyLength > keyBuffer.length) { + // This is a killer. We need to provide sufficient space for the buffer + if (keyLength + keyBuffer.position() > keyBuffer.capacity()) { // rare case, but we need to handle arbitrary length of key - int newKeyBufferLength = Math.max(keyBuffer.length, 1) * 2; - while (keyLength > newKeyBufferLength) { + int newKeyBufferLength = Math.max(keyBuffer.capacity(), 1) * 2; + while (keyLength + keyBuffer.position() > newKeyBufferLength) { newKeyBufferLength *= 2; } - byte[] newKeyBuffer = new byte[newKeyBufferLength]; - System.arraycopy(keyBuffer, 0, newKeyBuffer, 0, keyBuffer.length); + ByteBuffer newKeyBuffer = ByteBuffer.allocate(newKeyBufferLength); + System.arraycopy(keyBuffer.array(), 0, newKeyBuffer.array(), 0, keyBuffer.position()); + newKeyBuffer.position(keyBuffer.position()); keyBuffer = newKeyBuffer; } } - protected void ensureSpaceForTags() { if (tagsLength > tagsBuffer.length) { // rare case, but we need to handle arbitrary length of tags @@ -133,28 +138,16 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { * seeker to the next key/value. */ protected void copyFromNext(SeekerState nextState) { - if (keyBuffer.length != nextState.keyBuffer.length) { - keyBuffer = nextState.keyBuffer.clone(); - } else if (!isValid()) { - // Note: we can only call isValid before we override our state, so this - // comes before all the assignments at the end of this method. - System.arraycopy(nextState.keyBuffer, 0, keyBuffer, 0, - nextState.keyLength); - } else { - // don't copy the common prefix between this key and the previous one - System.arraycopy(nextState.keyBuffer, nextState.lastCommonPrefix, - keyBuffer, nextState.lastCommonPrefix, nextState.keyLength - - nextState.lastCommonPrefix); - } - + // No key copy would be needed here valueOffset = nextState.valueOffset; keyLength = nextState.keyLength; valueLength = nextState.valueLength; lastCommonPrefix = nextState.lastCommonPrefix; nextKvOffset = nextState.nextKvOffset; memstoreTS = nextState.memstoreTS; + keyOffset = nextState.keyOffset; + keyBuffer = nextState.keyBuffer; } - } protected abstract static class @@ -192,14 +185,15 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public int compareKey(KVComparator comparator, byte[] key, int offset, int length) { - return comparator.compareFlatKey(key, offset, length, - current.keyBuffer, 0, current.keyLength); + return comparator.compareFlatKey(key, offset, length, current.keyBuffer.array(), + current.keyBuffer.arrayOffset() + current.keyOffset, current.keyLength); } @Override public int compareKey(KVComparator comparator, Cell key) { return comparator.compareOnlyKeyPortion(key, - new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength)); + new KeyValue.KeyOnlyKeyValue(current.keyBuffer.array(), + current.keyOffset, current.keyLength)); } @Override @@ -208,6 +202,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { this.tagCompressionContext.clear(); } currentBuffer = buffer; + // Allocate every time we get a new block + // Would be great if from the block we know how much is key part and how + // much is for value part(the unencoded one). If this value exceeds we + // may need to do a copy + // TODO : Get the unencoded key length from the hfileblock + current.keyBuffer = ByteBuffer.allocate(currentBuffer.capacity() * 16); decodeFirst(); previous.invalidate(); } @@ -215,7 +215,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public ByteBuffer getKeyDeepCopy() { ByteBuffer keyBuffer = ByteBuffer.allocate(current.keyLength); - keyBuffer.put(current.keyBuffer, 0, current.keyLength); + keyBuffer.put(current.keyBuffer.array(), current.keyOffset, + current.keyLength); return keyBuffer; } @@ -231,7 +232,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { ByteBuffer kvBuffer = createKVBuffer(); kvBuffer.putInt(current.keyLength); kvBuffer.putInt(current.valueLength); - kvBuffer.put(current.keyBuffer, 0, current.keyLength); + kvBuffer.put(current.keyBuffer.array(), current.keyOffset, + current.keyLength); kvBuffer.put(currentBuffer.array(), currentBuffer.arrayOffset() + current.valueOffset, current.valueLength); @@ -259,7 +261,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } @Override - public Cell getKeyValue() { + public Cell getCell() { ByteBuffer kvBuf = getKeyValueBuffer(); KeyValue kv = new KeyValue(kvBuf.array(), kvBuf.arrayOffset(), kvBuf.array().length - kvBuf.arrayOffset()); @@ -273,6 +275,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { if (tagCompressionContext != null) { tagCompressionContext.clear(); } + current.keyBuffer.rewind(); decodeFirst(); previous.invalidate(); } @@ -327,7 +330,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { do { int comp; if (samePrefixComparator != null) { - currentCell.setKey(current.keyBuffer, 0, current.keyLength); + currentCell.setKey(current.keyBuffer.array(), + + current.keyOffset, current.keyLength); if (current.lastCommonPrefix != 0) { // The KV format has row key length also in the byte array. The // common prefix @@ -383,9 +387,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } } } - } + } } else { - Cell r = new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength); + Cell r = new KeyValue.KeyOnlyKeyValue(current.keyBuffer.array(), + current.keyOffset, current.keyLength); comp = comparator.compareOnlyKeyPortion(seekCell, r); } if (comp == 0) { // exact match diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java index 1dc8413..6fa5f68 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java @@ -60,10 +60,16 @@ public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder { return new BufferedEncodedSeeker(comparator, decodingCtx) { @Override protected void decodeNext() { + int prevKeyOffset = current.keyOffset; + current.keyOffset = current.keyBuffer.arrayOffset() + current.keyBuffer.position(); current.keyLength = currentBuffer.getInt(); current.valueLength = currentBuffer.getInt(); current.ensureSpaceForKey(); - currentBuffer.get(current.keyBuffer, 0, current.keyLength); + currentBuffer.get(current.keyBuffer.array(), + + current.keyOffset, current.keyLength); + // Check the indices here + current.keyBuffer.position(current.keyOffset + + + current.keyLength); current.valueOffset = currentBuffer.position(); ByteBufferUtils.skip(currentBuffer, current.valueLength); if (includesTags()) { diff --git 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 index ddb2359..2ec3b13 100644 --- 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 @@ -143,10 +143,10 @@ public interface DataBlockEncoder { ByteBuffer getKeyValueBuffer(); /** - * @return the KeyValue object at the current position. Includes memstore + * @return the Cell at the current position. Includes memstore * timestamp. */ - Cell getKeyValue(); + Cell getCell(); /** Set position to beginning of given block */ void rewind(); diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java index f72878b..8dae3c7 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java @@ -410,12 +410,16 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE; private void decode(boolean isFirst) { + int prevKeyOffset = current.keyOffset; + current.keyOffset = current.keyBuffer.arrayOffset() + current.keyBuffer.position(); + int position = current.keyOffset; byte flag = currentBuffer.get(); byte type = 0; + if (!isFirst) { + type = current.keyBuffer.array()[prevKeyOffset + + current.keyLength - Bytes.SIZEOF_BYTE]; + } if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - if (!isFirst) { - type = current.keyBuffer[current.keyLength - Bytes.SIZEOF_BYTE]; - } current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); } if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { @@ -424,53 +428,73 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { } current.lastCommonPrefix = ByteBufferUtils.readCompressedInt(currentBuffer); - current.ensureSpaceForKey(); - if (current.lastCommonPrefix < Bytes.SIZEOF_SHORT) { // length of row is different, copy everything except family - + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the row size - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, + currentBuffer.get(current.keyBuffer.array(), + position, Bytes.SIZEOF_SHORT - current.lastCommonPrefix); - current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - + current.keyBuffer.position(position + Bytes.SIZEOF_SHORT - current.lastCommonPrefix); + current.rowLengthWithSize = current.keyBuffer.getShort(current.keyBuffer.position() + - Bytes.SIZEOF_SHORT) + + Bytes.SIZEOF_SHORT; + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the rest of row - currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT, - current.rowLengthWithSize - Bytes.SIZEOF_SHORT); - + currentBuffer.get(current.keyBuffer.array(), position + , current.rowLengthWithSize - Bytes.SIZEOF_SHORT); + current.keyBuffer.position(position + + current.rowLengthWithSize - Bytes.SIZEOF_SHORT); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the column family - System.arraycopy(familyNameWithSize, 0, current.keyBuffer, - current.rowLengthWithSize, familyNameWithSize.length); - + current.keyBuffer.put(familyNameWithSize, 0, familyNameWithSize.length); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the qualifier - currentBuffer.get(current.keyBuffer, - current.rowLengthWithSize + familyNameWithSize.length, + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength - current.rowLengthWithSize - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); + current.keyBuffer.position(position + current.keyLength - current.rowLengthWithSize - + familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } else if (current.lastCommonPrefix < current.rowLengthWithSize) { // we have to copy part of row and qualifier, // but column family is in right place // before column family (rest of row) - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.rowLengthWithSize - current.lastCommonPrefix); - + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + + currentBuffer.get(current.keyBuffer.array(), position, current.rowLengthWithSize + - current.lastCommonPrefix); + current.keyBuffer.position(position + current.rowLengthWithSize + - current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + current.keyBuffer.put(familyNameWithSize, 0, familyNameWithSize.length); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // after column family (qualifier) - currentBuffer.get(current.keyBuffer, - current.rowLengthWithSize + familyNameWithSize.length, - current.keyLength - current.rowLengthWithSize - - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength + - current.rowLengthWithSize - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); + current.keyBuffer.position(position + current.keyLength - current.rowLengthWithSize + - familyNameWithSize.length - TIMESTAMP_WITH_TYPE_LENGTH); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } else { // copy just the ending - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength - TIMESTAMP_WITH_TYPE_LENGTH - current.lastCommonPrefix); + current.keyBuffer.position(position + current.keyLength - TIMESTAMP_WITH_TYPE_LENGTH + - current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } // timestamp - int pos = current.keyLength - TIMESTAMP_WITH_TYPE_LENGTH; int timestampFitInBytes = 1 + ((flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH); long timestampOrDiff = @@ -483,16 +507,20 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { } else { // it is diff current.timestamp = current.timestamp - timestampOrDiff; } - Bytes.putLong(current.keyBuffer, pos, current.timestamp); - pos += Bytes.SIZEOF_LONG; + current.keyBuffer.putLong(position, current.timestamp); + current.keyBuffer.position(position + Bytes.SIZEOF_LONG); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // type if ((flag & FLAG_SAME_TYPE) == 0) { - currentBuffer.get(current.keyBuffer, pos, Bytes.SIZEOF_BYTE); + currentBuffer.get(current.keyBuffer.array(), position, Bytes.SIZEOF_BYTE); + current.keyBuffer.position(position + Bytes.SIZEOF_BYTE); } else if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - current.keyBuffer[pos] = type; + current.keyBuffer.put(type); + } else { + current.keyBuffer.put(type); } - + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); current.valueOffset = currentBuffer.position(); ByteBufferUtils.skip(currentBuffer, current.valueLength); diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java index 0346b20..553da77 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java @@ -420,14 +420,17 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { final HFileBlockDecodingContext decodingCtx) { return new BufferedEncodedSeeker(comparator, decodingCtx) { private void decode(boolean isFirst) { + int prevKeyOffset = current.keyOffset; + current.keyOffset = current.keyBuffer.arrayOffset() + current.keyBuffer.position(); + int position = current.keyOffset; byte flag = currentBuffer.get(); + if (!isFirst) { + // Need to copy this + System.arraycopy(current.keyBuffer.array(), prevKeyOffset + current.keyLength + - current.prevTimestampAndType.length, current.prevTimestampAndType, 0, + current.prevTimestampAndType.length); + } if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - if (!isFirst) { - System.arraycopy(current.keyBuffer, - current.keyLength - current.prevTimestampAndType.length, - current.prevTimestampAndType, 0, - current.prevTimestampAndType.length); - } current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); } if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) { @@ -436,82 +439,112 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { } current.lastCommonPrefix = ByteBufferUtils.readCompressedInt(currentBuffer); - current.ensureSpaceForKey(); - if (isFirst) { // copy everything - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, + currentBuffer.get(current.keyBuffer.array(), position + current.lastCommonPrefix, current.keyLength - current.prevTimestampAndType.length); - current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - current.familyLengthWithSize = - current.keyBuffer[current.rowLengthWithSize] + Bytes.SIZEOF_BYTE; + current.keyBuffer.position(position + current.lastCommonPrefix + + current.keyLength - current.prevTimestampAndType.length); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + current.rowLengthWithSize = current.keyBuffer.getShort(0) + + Bytes.SIZEOF_SHORT; + current.familyLengthWithSize = current.keyBuffer.get(current.rowLengthWithSize) + + Bytes.SIZEOF_BYTE; } else if (current.lastCommonPrefix < Bytes.SIZEOF_SHORT) { // length of row is different, copy everything except family - + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the row size int oldRowLengthWithSize = current.rowLengthWithSize; - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - Bytes.SIZEOF_SHORT - current.lastCommonPrefix); - current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) + - Bytes.SIZEOF_SHORT; - - // move the column family - System.arraycopy(current.keyBuffer, oldRowLengthWithSize, - current.keyBuffer, current.rowLengthWithSize, - current.familyLengthWithSize); + currentBuffer.get(current.keyBuffer.array(), position, Bytes.SIZEOF_SHORT + - current.lastCommonPrefix); + current.keyBuffer.position(position + Bytes.SIZEOF_SHORT - current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + current.rowLengthWithSize = current.keyBuffer.getShort(position - Bytes.SIZEOF_SHORT) + + Bytes.SIZEOF_SHORT; // copy the rest of row - currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT, - current.rowLengthWithSize - Bytes.SIZEOF_SHORT); + currentBuffer.get(current.keyBuffer.array(), position, current.rowLengthWithSize + - Bytes.SIZEOF_SHORT); + current.keyBuffer.position(position + current.rowLengthWithSize - Bytes.SIZEOF_SHORT); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + // move the column family + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset + + oldRowLengthWithSize, current.familyLengthWithSize); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // copy the qualifier - currentBuffer.get(current.keyBuffer, current.rowLengthWithSize - + current.familyLengthWithSize, current.keyLength + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength - current.rowLengthWithSize - current.familyLengthWithSize - current.prevTimestampAndType.length); + current.keyBuffer.position(position + current.keyLength - current.rowLengthWithSize + - current.familyLengthWithSize - current.prevTimestampAndType.length); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } else if (current.lastCommonPrefix < current.rowLengthWithSize) { // We have to copy part of row and qualifier, but the column family // is in the right place. // before column family (rest of row) - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, - current.rowLengthWithSize - current.lastCommonPrefix); - + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + + currentBuffer.get(current.keyBuffer.array(), position, current.rowLengthWithSize + - current.lastCommonPrefix); + current.keyBuffer.position(position + current.rowLengthWithSize + - current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset + + current.rowLengthWithSize, current.familyLengthWithSize); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // after column family (qualifier) - currentBuffer.get(current.keyBuffer, current.rowLengthWithSize - + current.familyLengthWithSize, current.keyLength + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength - current.rowLengthWithSize - current.familyLengthWithSize - current.prevTimestampAndType.length); + current.keyBuffer.position(position + current.keyLength - current.rowLengthWithSize + - current.familyLengthWithSize - current.prevTimestampAndType.length); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } else { // copy just the ending - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + currentBuffer.get(current.keyBuffer.array(), position, current.keyLength - current.prevTimestampAndType.length - current.lastCommonPrefix); + current.keyBuffer.position(position + + current.keyLength - current.prevTimestampAndType.length + - current.lastCommonPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); } // timestamp - int pos = current.keyLength - current.prevTimestampAndType.length; int commonTimestampPrefix = (flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH; if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - System.arraycopy(current.prevTimestampAndType, 0, current.keyBuffer, - pos, commonTimestampPrefix); + // TODO : code repeating here + current.keyBuffer.put(current.prevTimestampAndType, 0, commonTimestampPrefix); + } else { + current.keyBuffer.put(current.prevTimestampAndType, 0, commonTimestampPrefix); } - pos += commonTimestampPrefix; - currentBuffer.get(current.keyBuffer, pos, + current.keyBuffer.position(position + commonTimestampPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + currentBuffer.get(current.keyBuffer.array(), position, Bytes.SIZEOF_LONG - commonTimestampPrefix); - pos += Bytes.SIZEOF_LONG - commonTimestampPrefix; - + current.keyBuffer.position(position + Bytes.SIZEOF_LONG - commonTimestampPrefix); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // type if ((flag & FLAG_SAME_TYPE) == 0) { - currentBuffer.get(current.keyBuffer, pos, Bytes.SIZEOF_BYTE); + currentBuffer.get(current.keyBuffer.array(), position, Bytes.SIZEOF_BYTE); } else if ((flag & FLAG_SAME_KEY_LENGTH) == 0) { - current.keyBuffer[pos] = - current.prevTimestampAndType[Bytes.SIZEOF_LONG]; + current.keyBuffer.put(current.prevTimestampAndType[Bytes.SIZEOF_LONG]); + } else { + current.keyBuffer.put(current.prevTimestampAndType[Bytes.SIZEOF_LONG]); } - + current.keyBuffer.position(position + Bytes.SIZEOF_BYTE); + position = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); // handle value if ((flag & FLAG_SAME_VALUE) == 0) { current.valueOffset = currentBuffer.position(); diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java index f57ff4f..3c94403 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java @@ -168,14 +168,23 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { return new BufferedEncodedSeeker(comparator, decodingCtx) { @Override protected void decodeNext() { + int prevKeyOffset = current.keyOffset; + current.keyOffset = current.keyBuffer.arrayOffset() + current.keyBuffer.position(); current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer); current.valueLength = ByteBufferUtils.readCompressedInt(currentBuffer); - current.lastCommonPrefix = - ByteBufferUtils.readCompressedInt(currentBuffer); + current.lastCommonPrefix = ByteBufferUtils.readCompressedInt(currentBuffer); current.keyLength += current.lastCommonPrefix; current.ensureSpaceForKey(); - currentBuffer.get(current.keyBuffer, current.lastCommonPrefix, + int offset = current.keyOffset; + if (current.lastCommonPrefix != 0) { + current.keyBuffer.put(current.keyBuffer.array(), prevKeyOffset, + current.lastCommonPrefix); + offset = current.keyBuffer.position() + current.keyBuffer.arrayOffset(); + } + currentBuffer.get(current.keyBuffer.array(), offset, current.keyLength - current.lastCommonPrefix); + current.keyBuffer + .position(offset + current.keyLength - current.lastCommonPrefix); current.valueOffset = currentBuffer.position(); ByteBufferUtils.skip(currentBuffer, current.valueLength); if (includesTags()) { diff --git 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 index 65dd1ce..a7a1d70 100644 --- 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 @@ -91,7 +91,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { * currently must do deep copy into new array */ @Override - public KeyValue getKeyValue() { + public KeyValue getCell() { if (ptSearcher.current() == null) { return null; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java index 61e2bb5..72307d7 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java @@ -148,10 +148,10 @@ public class HalfStoreFileReader extends StoreFile.Reader { return delegate.getValueString(); } - public Cell getKeyValue() { + public Cell getCell() { if (atEnd) return null; - return delegate.getKeyValue(); + return delegate.getCell(); } public boolean next() throws IOException { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 642a33e..dcdf5dc 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -263,7 +263,7 @@ public class HFilePrettyPrinter { HFileScanner scanner, byte[] row) throws IOException { KeyValue pkv = null; do { - KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue()); + KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getCell()); if (row != null && row.length != 0) { int result = Bytes.compareTo(kv.getRow(), row); if (result > 0) { diff --git 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 index 3fbebb3..7769fdd 100644 --- 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 @@ -716,7 +716,7 @@ public class HFileReaderV2 extends AbstractHFileReader { } @Override - public Cell getKeyValue() { + public Cell getCell() { if (!isSeeked()) return null; @@ -1178,11 +1178,11 @@ public class HFileReaderV2 extends AbstractHFileReader { } @Override - public Cell getKeyValue() { + public Cell getCell() { if (block == null) { return null; } - return seeker.getKeyValue(); + return seeker.getCell(); } @Override diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java index b951fab..0d2783e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java @@ -23,7 +23,6 @@ import java.nio.ByteBuffer; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; /** * A scanner allows you to position yourself within a HFile and @@ -132,9 +131,9 @@ public interface HFileScanner { */ ByteBuffer getValue(); /** - * @return Instance of {@link KeyValue}. + * @return Instance of {@link Cell}. */ - Cell getKeyValue(); + Cell getCell(); /** * Convenience method to get a copy of the key as a string - interpreting the * bytes as UTF8. You must call {@link #seekTo(byte[])} before this method. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 4fb8420..0ef97da 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -743,7 +743,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { HFileScanner scanner = halfReader.getScanner(false, false, false); scanner.seekTo(); do { - KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue()); + KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getCell()); halfWriter.append(kv); } while (scanner.next()); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index cb5fbcb..03d79a7 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -637,7 +637,7 @@ public class HStore implements Store { HFileScanner scanner = reader.getScanner(false, false, false); scanner.seekTo(); do { - Cell kv = scanner.getKeyValue(); + Cell kv = scanner.getCell(); if (prevKV != null) { if (Bytes.compareTo(prevKV.getRowArray(), prevKV.getRowOffset(), prevKV.getRowLength(), kv.getRowArray(), kv.getRowOffset(), @@ -1619,7 +1619,7 @@ public class HStore implements Store { // If here, need to start backing up. while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(), firstOnRow.getKeyLength())) { - Cell kv = scanner.getKeyValue(); + Cell kv = scanner.getCell(); if (!state.isTargetTable(kv)) break; if (!state.isBetterCandidate(kv)) break; // Make new first on row. @@ -1667,7 +1667,7 @@ public class HStore implements Store { throws IOException { boolean foundCandidate = false; do { - Cell kv = scanner.getKeyValue(); + Cell kv = scanner.getCell(); // If we are not in the row, skip. if (this.comparator.compareRows(kv, firstOnRow) < 0) continue; // Did we go beyond the target row? If so break. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index f0f92ae..a23a51b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; * bloom filter things. */ @InterfaceAudience.LimitedPrivate("Coprocessor") -public class StoreFileScanner implements KeyValueScanner { +public class StoreFileScanner implements KeyValueScanner{ static final Log LOG = LogFactory.getLog(HStore.class); // the reader it comes from: @@ -137,7 +137,7 @@ public class StoreFileScanner implements KeyValueScanner { // only seek if we aren't at the end. cur == null implies 'end'. if (cur != null) { hfs.next(); - cur = hfs.getKeyValue(); + cur = hfs.getCell(); if (hasMVCCInfo) skipKVsNewerThanReadpoint(); } @@ -157,7 +157,7 @@ public class StoreFileScanner implements KeyValueScanner { return false; } - cur = hfs.getKeyValue(); + cur = hfs.getCell(); return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint(); } finally { @@ -177,7 +177,7 @@ public class StoreFileScanner implements KeyValueScanner { close(); return false; } - cur = hfs.getKeyValue(); + cur = hfs.getCell(); return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint(); } finally { @@ -197,7 +197,7 @@ public class StoreFileScanner implements KeyValueScanner { && cur != null && (cur.getMvccVersion() > readPt)) { hfs.next(); - cur = hfs.getKeyValue(); + cur = hfs.getCell(); if (this.stopSkippingKVsIfNextRow && getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(), cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(), @@ -420,8 +420,8 @@ public class StoreFileScanner implements KeyValueScanner { close(); return false; } - KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue() - .getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength()); + KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getCell() + .getRowArray(), hfs.getCell().getRowOffset(), hfs.getCell().getRowLength()); if (seekCount != null) seekCount.incrementAndGet(); if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) { @@ -429,7 +429,7 @@ public class StoreFileScanner implements KeyValueScanner { return false; } - cur = hfs.getKeyValue(); + cur = hfs.getCell(); this.stopSkippingKVsIfNextRow = true; boolean resultOfSkipKVs; try { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index eb0023e..586d6bc 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -125,7 +125,7 @@ public class TestHalfStoreFileReader { scanner.seekTo(); Cell curr; do { - curr = scanner.getKeyValue(); + curr = scanner.getCell(); KeyValue reseekKv = getLastOnCol(curr); int ret = scanner.reseekTo(reseekKv); @@ -223,7 +223,7 @@ public class TestHalfStoreFileReader { halfreader.loadFileInfo(); final HFileScanner scanner = halfreader.getScanner(false, false); scanner.seekBefore(seekBefore); - return scanner.getKeyValue(); + return scanner.getCell(); } private KeyValue getLastOnCol(Cell curr) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java index 31e3b14..6e27917 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java @@ -32,10 +32,10 @@ public class TestBufferedDataBlockEncoder { for (int i = 1; i <= 65536; ++i) { state.keyLength = i; state.ensureSpaceForKey(); - state.keyBuffer[state.keyLength - 1] = (byte) ((i - 1) % 0xff); + state.keyBuffer.array()[state.keyLength - 1] = (byte) ((i - 1) % 0xff); for (int j = 0; j < i - 1; ++j) { // Check that earlier bytes were preserved as the buffer grew. - assertEquals((byte) (j % 0xff), state.keyBuffer[j]); + assertEquals((byte) (j % 0xff), state.keyBuffer.array()[j]); } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java index 1ab4ca7..f40789d 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java @@ -119,7 +119,7 @@ public class TestPrefixTreeEncoding { seeker.seekToKeyInBlock( new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey .getKeyLength()), true); - assertEquals(null, seeker.getKeyValue()); + assertEquals(null, seeker.getCell()); // Seek before the middle keyvalue; seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3), @@ -127,16 +127,16 @@ public class TestPrefixTreeEncoding { seeker.seekToKeyInBlock( new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey .getKeyLength()), true); - assertNotNull(seeker.getKeyValue()); - assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3 - 1), seeker.getKeyValue().getRow()); + assertNotNull(seeker.getCell()); + assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3 - 1), seeker.getCell().getRow()); // Seek before the last keyvalue; seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES); seeker.seekToKeyInBlock( new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey .getKeyLength()), true); - assertNotNull(seeker.getKeyValue()); - assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH - 1), seeker.getKeyValue().getRow()); + assertNotNull(seeker.getCell()); + assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH - 1), seeker.getCell().getRow()); } @Test @@ -160,7 +160,7 @@ public class TestPrefixTreeEncoding { seeker.setCurrentBuffer(readBuffer); Cell previousKV = null; do { - Cell currentKV = seeker.getKeyValue(); + Cell currentKV = seeker.getCell(); System.out.println(currentKV); if (previousKV != null && KeyValue.COMPARATOR.compare(currentKV, previousKV) < 0) { dumpInputKVSet(); @@ -238,11 +238,11 @@ public class TestPrefixTreeEncoding { fail("Get error result after seeking " + firstOnRow); } if (hasMoreOfEncodeScanner) { - if (KeyValue.COMPARATOR.compare(encodeSeeker.getKeyValue(), + if (KeyValue.COMPARATOR.compare(encodeSeeker.getCell(), collectionScanner.peek()) != 0) { dumpInputKVSet(); fail("Expected " + collectionScanner.peek() + " actual " - + encodeSeeker.getKeyValue() + ", after seeking " + firstOnRow); + + encodeSeeker.getCell() + ", after seeking " + firstOnRow); } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java index 75464d2..4e7f4c5 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; @@ -46,7 +47,7 @@ public class TestSeekToBlockWithEncoders { DataBlockEncoding encoding) { DataBlockEncoder encoder = encoding.getEncoder(); HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(false) - .withIncludesTags(false).withCompression(algo).build(); + .withIncludesTags(true).withCompression(algo).build(); if (encoder != null) { return encoder .newDataBlockEncodingContext(encoding, HConstants.HFILEBLOCK_DUMMY_HEADER, meta); @@ -63,19 +64,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithNonMatchingSeekKey() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv3); KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), @@ -90,19 +96,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekingToBlockWithBiggerNonLength1() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv3); KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aaaad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), @@ -117,19 +128,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekingToBlockToANotAvailableKey() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv3); KeyValue kv4 = new KeyValue(Bytes.toBytes("aaae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"), @@ -144,16 +160,20 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), - Bytes.toBytes("q1"), Bytes.toBytes("val")); + Bytes.toBytes("q1"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), - Bytes.toBytes("q2"), Bytes.toBytes("val")); + Bytes.toBytes("q2"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), - Bytes.toBytes("q3"), Bytes.toBytes("val")); + Bytes.toBytes("q3"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv3); KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"), - Bytes.toBytes("q1"), Bytes.toBytes("val")); + Bytes.toBytes("q1"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(), @@ -165,16 +185,21 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDiffQualifer() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); + ; sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), @@ -186,19 +211,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv6); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), @@ -210,19 +240,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv6); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"), @@ -234,19 +269,24 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv6); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"), @@ -258,16 +298,20 @@ public class TestSeekToBlockWithEncoders { public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException { List sampleKv = new ArrayList(); KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv1); KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv2); KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv4); KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"), - Bytes.toBytes("val")); + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("val"), + new Tag[] { new Tag((byte) 1, "value1") }); sampleKv.add(kv5); ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false); KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"), @@ -278,8 +322,7 @@ public class TestSeekToBlockWithEncoders { private void seekToTheKey(KeyValue expected, ByteBuffer originalBuffer, KeyValue toSeek) throws IOException { // create all seekers - List encodedSeekers = - new ArrayList(); + List encodedSeekers = new ArrayList(); for (DataBlockEncoding encoding : DataBlockEncoding.values()) { if (encoding.getEncoder() == null || encoding == DataBlockEncoding.PREFIX_TREE) { continue; @@ -288,7 +331,7 @@ public class TestSeekToBlockWithEncoders { ByteBuffer encodedBuffer = ByteBuffer.wrap(encodeBytes(encoding, originalBuffer)); DataBlockEncoder encoder = encoding.getEncoder(); HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false) - .withIncludesMvcc(false).withIncludesTags(false) + .withIncludesMvcc(false).withIncludesTags(true) .withCompression(Compression.Algorithm.NONE).build(); DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR, encoder.newDataBlockDecodingContext(meta)); @@ -306,7 +349,7 @@ public class TestSeekToBlockWithEncoders { seeker.seekToKeyInBlock( new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue .getKeyLength()), false); - Cell keyValue2 = seeker.getKeyValue(); + Cell keyValue2 = seeker.getCell(); assertEquals(expected, keyValue2); seeker.rewind(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java index 31546e2..4d04c0e 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java @@ -222,7 +222,7 @@ public class TestHFileEncryption { assertTrue("Initial seekTo failed", scanner.seekTo()); int i = 0; do { - Cell kv = scanner.getKeyValue(); + Cell kv = scanner.getCell(); assertTrue("Read back an unexpected or invalid KV", testKvs.contains(KeyValueUtil.ensureKeyValue(kv))); i++; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java index 9297e64..b409707 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java @@ -112,27 +112,27 @@ public class TestSeekTo extends HBaseTestCase { assertEquals(false, scanner.seekBefore(toKV("c", tagUsage))); assertEquals(true, scanner.seekBefore(toKV("d", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("e", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("f", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("g", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("h", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("i", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("j", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("k", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); assertEquals(true, scanner.seekBefore(toKV("l", tagUsage))); - assertEquals("k", toRowStr(scanner.getKeyValue())); + assertEquals("k", toRowStr(scanner.getCell())); reader.close(); } @@ -154,76 +154,76 @@ public class TestSeekTo extends HBaseTestCase { // seekBefore d, so the scanner points to c assertEquals(true, scanner.seekBefore(toKV("d", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); // reseekTo e and g assertEquals(0, scanner.reseekTo(toKV("c", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore e, so the scanner points to c assertEquals(true, scanner.seekBefore(toKV("e", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); // reseekTo e and g assertEquals(0, scanner.reseekTo(toKV("e", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore f, so the scanner points to e assertEquals(true, scanner.seekBefore(toKV("f", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); // reseekTo e and g assertEquals(0, scanner.reseekTo(toKV("e", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore g, so the scanner points to e assertEquals(true, scanner.seekBefore(toKV("g", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); // reseekTo e and g again assertEquals(0, scanner.reseekTo(toKV("e", tagUsage))); - assertEquals("e", toRowStr(scanner.getKeyValue())); + assertEquals("e", toRowStr(scanner.getCell())); assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore h, so the scanner points to g assertEquals(true, scanner.seekBefore(toKV("h", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // reseekTo g assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore i, so the scanner points to g assertEquals(true, scanner.seekBefore(toKV("i", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // reseekTo g assertEquals(0, scanner.reseekTo(toKV("g", tagUsage))); - assertEquals("g", toRowStr(scanner.getKeyValue())); + assertEquals("g", toRowStr(scanner.getCell())); // seekBefore j, so the scanner points to i assertEquals(true, scanner.seekBefore(toKV("j", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); // reseekTo i assertEquals(0, scanner.reseekTo(toKV("i", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); // seekBefore k, so the scanner points to i assertEquals(true, scanner.seekBefore(toKV("k", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); // reseekTo i and k assertEquals(0, scanner.reseekTo(toKV("i", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); assertEquals(0, scanner.reseekTo(toKV("k", tagUsage))); - assertEquals("k", toRowStr(scanner.getKeyValue())); + assertEquals("k", toRowStr(scanner.getCell())); // seekBefore l, so the scanner points to k assertEquals(true, scanner.seekBefore(toKV("l", tagUsage))); - assertEquals("k", toRowStr(scanner.getKeyValue())); + assertEquals("k", toRowStr(scanner.getCell())); // reseekTo k assertEquals(0, scanner.reseekTo(toKV("k", tagUsage))); - assertEquals("k", toRowStr(scanner.getKeyValue())); + assertEquals("k", toRowStr(scanner.getCell())); } public void testSeekTo() throws Exception { @@ -242,15 +242,15 @@ public class TestSeekTo extends HBaseTestCase { assertEquals(-1, scanner.seekTo(toKV("a", tagUsage))); assertEquals(1, scanner.seekTo(toKV("d", tagUsage))); - assertEquals("c", toRowStr(scanner.getKeyValue())); + assertEquals("c", toRowStr(scanner.getCell())); // Across a block boundary now. // h goes to the next block assertEquals(-2, scanner.seekTo(toKV("h", tagUsage))); - assertEquals("i", toRowStr(scanner.getKeyValue())); + assertEquals("i", toRowStr(scanner.getCell())); assertEquals(1, scanner.seekTo(toKV("l", tagUsage))); - assertEquals("k", toRowStr(scanner.getKeyValue())); + assertEquals("k", toRowStr(scanner.getCell())); reader.close(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 262e421..58f74a4 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -356,7 +356,7 @@ public class TestMajorCompaction { HFileScanner scanner = f.getReader().getScanner(false, false); scanner.seekTo(); do { - byte [] row = scanner.getKeyValue().getRow(); + byte [] row = scanner.getCell().getRow(); if (Bytes.equals(row, STARTROW)) { count1++; } else if(Bytes.equals(row, secondRowBytes)) {