HBASE-18026 ProtobufUtil seems to do extra array copying
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
0ae0edcd63
commit
5895631634
|
@ -17,6 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.protobuf;
|
||||
|
||||
import static com.google.protobuf.HBaseZeroCopyByteString.zeroCopyGetBytes;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.Method;
|
||||
|
@ -367,7 +369,7 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static Get toGet(final ClientProtos.Get proto) throws IOException {
|
||||
if (proto == null) return null;
|
||||
byte[] row = proto.getRow().toByteArray();
|
||||
byte[] row = zeroCopyGetBytes(proto.getRow());
|
||||
Get get = new Get(row);
|
||||
if (proto.hasCacheBlocks()) {
|
||||
get.setCacheBlocks(proto.getCacheBlocks());
|
||||
|
@ -464,7 +466,7 @@ public final class ProtobufUtil {
|
|||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.PUT: type.name();
|
||||
long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
|
||||
Put put = proto.hasRow() ? new Put(proto.getRow().toByteArray(), timestamp) : null;
|
||||
Put put = proto.hasRow() ? new Put(zeroCopyGetBytes(proto.getRow()), timestamp) : null;
|
||||
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
|
||||
if (cellCount > 0) {
|
||||
// The proto has metadata only and the data is separate to be found in the cellScanner.
|
||||
|
@ -489,7 +491,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
// The proto has the metadata and the data itself
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
byte[] family = zeroCopyGetBytes(column.getFamily());
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
|
@ -508,7 +510,7 @@ public final class ProtobufUtil {
|
|||
allTagsBytes = qv.getTags().toByteArray();
|
||||
if(qv.hasDeleteType()) {
|
||||
byte[] qual = qv.hasQualifier() ? qv.getQualifier().toByteArray() : null;
|
||||
put.add(new KeyValue(proto.getRow().toByteArray(), family, qual, ts,
|
||||
put.add(new KeyValue(zeroCopyGetBytes(proto.getRow()), family, qual, ts,
|
||||
fromDeleteType(qv.getDeleteType()), null, allTagsBytes));
|
||||
} else {
|
||||
List<Tag> tags = TagUtil.asList(allTagsBytes, 0, (short)allTagsBytes.length);
|
||||
|
@ -517,8 +519,8 @@ public final class ProtobufUtil {
|
|||
}
|
||||
} else {
|
||||
if(qv.hasDeleteType()) {
|
||||
byte[] qual = qv.hasQualifier() ? qv.getQualifier().toByteArray() : null;
|
||||
put.add(new KeyValue(proto.getRow().toByteArray(), family, qual, ts,
|
||||
byte[] qual = qv.hasQualifier() ? zeroCopyGetBytes(qv.getQualifier()) : null;
|
||||
put.add(new KeyValue(zeroCopyGetBytes(proto.getRow()), family, qual, ts,
|
||||
fromDeleteType(qv.getDeleteType())));
|
||||
} else{
|
||||
put.addImmutable(family, qualifier, ts, value);
|
||||
|
@ -559,7 +561,7 @@ public final class ProtobufUtil {
|
|||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.DELETE : type.name();
|
||||
long timestamp = proto.hasTimestamp() ? proto.getTimestamp() : HConstants.LATEST_TIMESTAMP;
|
||||
Delete delete = proto.hasRow() ? new Delete(proto.getRow().toByteArray(), timestamp) : null;
|
||||
Delete delete = proto.hasRow() ? new Delete(zeroCopyGetBytes(proto.getRow()), timestamp) : null;
|
||||
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
|
||||
if (cellCount > 0) {
|
||||
// The proto has metadata only and the data is separate to be found in the cellScanner.
|
||||
|
@ -627,7 +629,7 @@ public final class ProtobufUtil {
|
|||
throws IOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.APPEND : type.name();
|
||||
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
|
||||
byte [] row = proto.hasRow()? zeroCopyGetBytes(proto.getRow()): null;
|
||||
Append append = null;
|
||||
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
|
||||
if (cellCount > 0) {
|
||||
|
@ -650,17 +652,17 @@ public final class ProtobufUtil {
|
|||
} else {
|
||||
append = new Append(row);
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
byte[] family = zeroCopyGetBytes(column.getFamily());
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
byte[] qualifier = zeroCopyGetBytes(qv.getQualifier());
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifier value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
byte[] value = zeroCopyGetBytes(qv.getValue());
|
||||
byte[] tags = null;
|
||||
if (qv.hasTags()) {
|
||||
tags = qv.getTags().toByteArray();
|
||||
tags = zeroCopyGetBytes(qv.getTags());
|
||||
}
|
||||
append.add(CellUtil.createCell(row, family, qualifier, qv.getTimestamp(),
|
||||
KeyValue.Type.Put, value, tags));
|
||||
|
@ -706,7 +708,7 @@ public final class ProtobufUtil {
|
|||
throws IOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.INCREMENT : type.name();
|
||||
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
|
||||
byte [] row = proto.hasRow()? zeroCopyGetBytes(proto.getRow()): null;
|
||||
Increment increment = null;
|
||||
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
|
||||
if (cellCount > 0) {
|
||||
|
@ -729,16 +731,16 @@ public final class ProtobufUtil {
|
|||
} else {
|
||||
increment = new Increment(row);
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
byte[] family = zeroCopyGetBytes(column.getFamily());
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
byte[] qualifier = zeroCopyGetBytes(qv.getQualifier());
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException("Missing required field: qualifier value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
byte[] value = zeroCopyGetBytes(qv.getValue());
|
||||
byte[] tags = null;
|
||||
if (qv.hasTags()) {
|
||||
tags = qv.getTags().toByteArray();
|
||||
tags = zeroCopyGetBytes(qv.getTags());
|
||||
}
|
||||
increment.add(CellUtil.createCell(row, family, qualifier, qv.getTimestamp(),
|
||||
KeyValue.Type.Put, value, tags));
|
||||
|
@ -767,7 +769,7 @@ public final class ProtobufUtil {
|
|||
throws IOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.INCREMENT || type == MutationType.APPEND : type.name();
|
||||
byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null;
|
||||
byte[] row = proto.hasRow() ? zeroCopyGetBytes(proto.getRow()) : null;
|
||||
Get get = null;
|
||||
int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0;
|
||||
if (cellCount > 0) {
|
||||
|
@ -793,9 +795,9 @@ public final class ProtobufUtil {
|
|||
} else {
|
||||
get = new Get(row);
|
||||
for (ColumnValue column : proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
byte[] family = zeroCopyGetBytes(column.getFamily());
|
||||
for (QualifierValue qv : column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
byte[] qualifier = zeroCopyGetBytes(qv.getQualifier());
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException("Missing required field: qualifier value");
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue