Index: hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java (revision 1543282) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java (working copy) @@ -69,7 +69,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; +import com.google.protobuf.ZeroCopyLiteralByteString; /** * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job @@ -157,7 +157,7 @@ MapReduceProtos.TableSnapshotRegionSplit.newBuilder() .setRegion(RegionSpecifier.newBuilder() .setType(RegionSpecifierType.ENCODED_REGION_NAME) - .setValue(ByteString.copyFrom(Bytes.toBytes(regionName))).build()); + .setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(regionName))).build()); for (String location : locations) { builder.addLocations(location); Index: hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java (revision 1543282) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java (working copy) @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import com.google.protobuf.ByteString; +import com.google.protobuf.ZeroCopyLiteralByteString; /** * Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp. @@ -45,15 +46,15 @@ CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder(); // This copies bytes from Cell to ByteString. I don't see anyway around the copy. // ByteString is final. - builder.setRow(ByteString.copyFrom(cell.getRowArray(), cell.getRowOffset(), + builder.setRow(ZeroCopyLiteralByteString.wrap(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); - builder.setFamily(ByteString.copyFrom(cell.getFamilyArray(), cell.getFamilyOffset(), + builder.setFamily(ZeroCopyLiteralByteString.wrap(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength())); - builder.setQualifier(ByteString.copyFrom(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength())); + builder.setQualifier(ZeroCopyLiteralByteString.wrap(cell.getQualifierArray(), + cell.getQualifierOffset(), cell.getQualifierLength())); builder.setTimestamp(cell.getTimestamp()); builder.setCellType(CellProtos.CellType.valueOf(cell.getTypeByte())); - builder.setValue(ByteString.copyFrom(cell.getValueArray(), cell.getValueOffset(), + builder.setValue(ZeroCopyLiteralByteString.wrap(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); CellProtos.Cell pbcell = builder.build(); pbcell.writeDelimitedTo(this.out);