diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index c6d647e7b8a..4e75694927f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ByteBufferCell; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -2077,19 +2078,40 @@ public final class ProtobufUtil { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); - kvbuilder.setRow(UnsafeByteOperations.unsafeWrap(kv.getRowArray(), kv.getRowOffset(), - kv.getRowLength())); - kvbuilder.setFamily(UnsafeByteOperations.unsafeWrap(kv.getFamilyArray(), - kv.getFamilyOffset(), kv.getFamilyLength())); - kvbuilder.setQualifier(UnsafeByteOperations.unsafeWrap(kv.getQualifierArray(), + if (kv instanceof ByteBufferCell) { + kvbuilder.setRow(wrap(((ByteBufferCell) kv).getRowByteBuffer(), + ((ByteBufferCell) kv).getRowPosition(), kv.getRowLength())); + kvbuilder.setFamily(wrap(((ByteBufferCell) kv).getFamilyByteBuffer(), + ((ByteBufferCell) kv).getFamilyPosition(), kv.getFamilyLength())); + kvbuilder.setQualifier(wrap(((ByteBufferCell) kv).getQualifierByteBuffer(), + ((ByteBufferCell) kv).getQualifierPosition(), kv.getQualifierLength())); + kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); + kvbuilder.setTimestamp(kv.getTimestamp()); + kvbuilder.setValue(wrap(((ByteBufferCell) kv).getValueByteBuffer(), + ((ByteBufferCell) kv).getValuePosition(), kv.getValueLength())); + // TODO : Once tags become first class then we may have to set tags to kvbuilder. + } else { + kvbuilder.setRow( + UnsafeByteOperations.unsafeWrap(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength())); + kvbuilder.setFamily(UnsafeByteOperations.unsafeWrap(kv.getFamilyArray(), kv.getFamilyOffset(), + kv.getFamilyLength())); + kvbuilder.setQualifier(UnsafeByteOperations.unsafeWrap(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); - kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); - kvbuilder.setTimestamp(kv.getTimestamp()); - kvbuilder.setValue(UnsafeByteOperations.unsafeWrap(kv.getValueArray(), kv.getValueOffset(), + kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); + kvbuilder.setTimestamp(kv.getTimestamp()); + kvbuilder.setValue(UnsafeByteOperations.unsafeWrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); + } return kvbuilder.build(); } + private static ByteString wrap(ByteBuffer b, int offset, int length) { + ByteBuffer dup = b.duplicate(); + dup.position(offset); + dup.limit(offset + length); + return UnsafeByteOperations.unsafeWrap(dup); + } + public static Cell toCell(final CellProtos.Cell cell) { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index b002b575793..c094ccb2988 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -19,10 +19,17 @@ package org.apache.hadoop.hbase.protobuf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.OffheapKeyValue; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; @@ -36,6 +43,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Col import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair; +import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -307,4 +315,25 @@ public class TestProtobufUtil { ProtobufUtil.toScan(expectedProto)); assertEquals(expectedProto, actualProto); } + + @Test + public void testToCell() throws Exception { + KeyValue kv1 = + new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]); + KeyValue kv2 = + new KeyValue(Bytes.toBytes("bbb"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]); + KeyValue kv3 = + new KeyValue(Bytes.toBytes("ccc"), Bytes.toBytes("f1"), Bytes.toBytes("q1"), new byte[30]); + byte[] arr = new byte[kv1.getLength() + kv2.getLength() + kv3.getLength()]; + System.arraycopy(kv1.getBuffer(), kv1.getOffset(), arr, 0, kv1.getLength()); + System.arraycopy(kv2.getBuffer(), kv2.getOffset(), arr, kv1.getLength(), kv2.getLength()); + System.arraycopy(kv3.getBuffer(), kv3.getOffset(), arr, kv1.getLength() + kv2.getLength(), + kv3.getLength()); + ByteBuffer dbb = ByteBuffer.allocateDirect(arr.length); + dbb.put(arr); + OffheapKeyValue offheapKV = new OffheapKeyValue(dbb, kv1.getLength(), kv2.getLength()); + CellProtos.Cell cell = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toCell(offheapKV); + Cell newOffheapKV = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toCell(cell); + assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0); + } }