diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index 94c7189..c07e71e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -712,8 +712,8 @@ public final class CellUtil { ((ByteBufferedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(), left.getFamilyOffset(), lfamlength); } - return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), - right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength()); + return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength, + right.getFamilyArray(), right.getFamilyOffset(), rfamlength); } public static boolean matchingFamily(final Cell left, final byte[] buf) { @@ -754,8 +754,8 @@ public final class CellUtil { left.getQualifierArray(), left.getQualifierOffset(), lqlength); } return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), - left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(), - right.getQualifierLength()); + lqlength, right.getQualifierArray(), right.getQualifierOffset(), + rqlength); } /** @@ -1638,8 +1638,8 @@ public final class CellUtil { ((ByteBufferedCell) right).getRowPosition(), rrowlength, left.getRowArray(), left.getRowOffset(), lrowlength); } - return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), - right.getRowArray(), right.getRowOffset(), right.getRowLength()); + return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, + right.getRowArray(), right.getRowOffset(), rrowlength); } /** diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index 43ae020..05ae4a2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -444,7 +444,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder { out.write(keyOnlyBuffer); // Write value out.write(this.valueBuffer, this.valueOffset, this.valueLength); - if (withTags) { + if (withTags && this.tagsLength > 0) { // 2 bytes tags length followed by tags bytes // tags length is serialized with 2 bytes only(short way) even if the type is int. // As this is non -ve numbers, we save the sign bit. See HBASE-11437 @@ -667,7 +667,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder { out.write(keyBuffer.array()); // Write value ByteBufferUtils.copyBufferToStream(out, this.valueBuffer, this.valueOffset, this.valueLength); - if (withTags) { + if (withTags && this.tagsLength > 0) { // 2 bytes tags length followed by tags bytes // tags length is serialized with 2 bytes only(short way) even if the type is int. // As this is non -ve numbers, we save the sign bit. See HBASE-11437 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 bd9715b..eee7344 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 @@ -573,9 +573,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return reader; } - protected int getCellBufSize() { + // From non encoded HFiles, we always read back KeyValue or its derives. (Note : When HFile + // block is in DBB, it will be OffheapKV). So all parts of the Cell is in a contiguous + // array/buffer. How many bytes we should wrap to make the KV is what this method returns. + private int getKVBufSize() { int kvBufSize = KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen; - if (this.reader.getFileContext().isIncludesTags()) { + if (currTagsLen > 0) { kvBufSize += Bytes.SIZEOF_SHORT + currTagsLen; } return kvBufSize; @@ -586,7 +589,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { this.returnBlocks(true); } - protected int getCurCellSize() { + // Returns the #bytes in HFile for the current cell. Used to skip these many bytes in current + // HFile block's buffer so as to position to the next cell. + private int getCurCellSerializedSize() { int curCellSize = KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen + currMemstoreTSLen; if (this.reader.getFileContext().isIncludesTags()) { @@ -934,7 +939,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return null; Cell ret; - int cellBufSize = getCellBufSize(); + int cellBufSize = getKVBufSize(); long seqId = 0l; if (this.reader.shouldIncludeMemstoreTS()) { seqId = currMemstoreTS; @@ -1015,7 +1020,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { */ private void positionThisBlockBuffer() { try { - blockBuffer.skip(getCurCellSize()); + blockBuffer.skip(getCurCellSerializedSize()); } catch (IllegalArgumentException e) { LOG.error("Current pos = " + blockBuffer.position() + "; currKeyLen = " + currKeyLen + "; currValLen = " diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java index d7e5bf5..d31af31 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java @@ -17,12 +17,22 @@ package org.apache.hadoop.hbase.io.encoding; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; +import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OffheapDecodedCell; +import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell; +import org.apache.hadoop.hbase.codec.Codec.Decoder; +import org.apache.hadoop.hbase.codec.Codec.Encoder; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -76,4 +86,48 @@ public class TestBufferedDataBlockEncoder { assertTrue((BufferedDataBlockEncoder.compareCommonQualifierPrefix(kv1, kv2, 4) > 0)); } + @Test + public void testKVCodecWithTagsForDecodedCellsWithNoTags() throws Exception { + KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"), + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1")); + // kv1.getKey() return a copy of the Key bytes which starts from RK_length. Means from offsets, + // we need to reduce the KL and VL parts. + OnheapDecodedCell c1 = new OnheapDecodedCell(kv1.getKey(), kv1.getRowLength(), + kv1.getFamilyOffset() - KeyValue.ROW_OFFSET, kv1.getFamilyLength(), + kv1.getQualifierOffset() - KeyValue.ROW_OFFSET, kv1.getQualifierLength(), + kv1.getTimestamp(), kv1.getTypeByte(), kv1.getValueArray(), kv1.getValueOffset(), + kv1.getValueLength(), kv1.getSequenceId(), kv1.getTagsArray(), kv1.getTagsOffset(), + kv1.getTagsLength()); + KeyValue kv2 = new KeyValue(Bytes.toBytes("r2"), Bytes.toBytes("f"), Bytes.toBytes("2"), + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2")); + OnheapDecodedCell c2 = new OnheapDecodedCell(kv2.getKey(), kv2.getRowLength(), + kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(), + kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(), + kv2.getTimestamp(), kv2.getTypeByte(), kv2.getValueArray(), kv2.getValueOffset(), + kv2.getValueLength(), kv2.getSequenceId(), kv2.getTagsArray(), kv2.getTagsOffset(), + kv2.getTagsLength()); + KeyValue kv3 = new KeyValue(Bytes.toBytes("r3"), Bytes.toBytes("cf"), Bytes.toBytes("qual"), + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3")); + OffheapDecodedCell c3 = new OffheapDecodedCell(ByteBuffer.wrap(kv2.getKey()), + kv2.getRowLength(), kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(), + kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(), + kv2.getTimestamp(), kv2.getTypeByte(), ByteBuffer.wrap(kv2.getValueArray()), + kv2.getValueOffset(), kv2.getValueLength(), kv2.getSequenceId(), + ByteBuffer.wrap(kv2.getTagsArray()), kv2.getTagsOffset(), kv2.getTagsLength()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + KeyValueCodecWithTags codec = new KeyValueCodecWithTags(); + Encoder encoder = codec.getEncoder(os); + encoder.write(c1); + encoder.write(c2); + encoder.write(c3); + ByteArrayInputStream is = new ByteArrayInputStream(os.toByteArray()); + Decoder decoder = codec.getDecoder(is); + assertTrue(decoder.advance()); + assertTrue(CellUtil.equals(c1, decoder.current())); + assertTrue(decoder.advance()); + assertTrue(CellUtil.equals(c2, decoder.current())); + assertTrue(decoder.advance()); + assertTrue(CellUtil.equals(c3, decoder.current())); + assertFalse(decoder.advance()); + } }