You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/11/01 00:10:15 UTC

svn commit: r1537756 [1/2] - in /hbase/branches/0.96: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ hbase-client/src...

Author: stack
Date: Thu Oct 31 23:10:13 2013
New Revision: 1537756

URL: http://svn.apache.org/r1537756
Log:
HBASE-9867 Save on array copies with a subclass of LiteralByteString

Added:
    hbase/branches/0.96/hbase-protocol/src/main/java/com/
    hbase/branches/0.96/hbase-protocol/src/main/java/com/google/
    hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/
    hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java
Modified:
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellSetModel.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/StorageClusterStatusModel.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableInfoModel.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
    hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java Thu Oct 31 23:10:13 2013
@@ -19,7 +19,13 @@
 
 package org.apache.hadoop.hbase;
 
-import com.google.protobuf.ByteString;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.master.RegionState;
@@ -34,12 +40,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.VersionedWritable;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 
 /**
@@ -334,7 +335,7 @@ public class ClusterStatus extends Versi
         ClusterStatusProtos.RegionState rs = rit.getValue().convert();
         RegionSpecifier.Builder spec =
             RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
-        spec.setValue(ByteString.copyFrom(Bytes.toBytes(rit.getKey())));
+        spec.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(rit.getKey())));
 
         RegionInTransition pbRIT =
             RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java Thu Oct 31 23:10:13 2013
@@ -19,8 +19,9 @@
 package org.apache.hadoop.hbase;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -1190,11 +1191,11 @@ public class HColumnDescriptor implement
    */
   public ColumnFamilySchema convert() {
     ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
-    builder.setName(ByteString.copyFrom(getName()));
+    builder.setName(ZeroCopyLiteralByteString.wrap(getName()));
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
       BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
-      aBuilder.setFirst(ByteString.copyFrom(e.getKey().get()));
-      aBuilder.setSecond(ByteString.copyFrom(e.getValue().get()));
+      aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get()));
+      aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get()));
       builder.addAttributes(aBuilder.build());
     }
     for (Map.Entry<String, String> e : this.configuration.entrySet()) {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java Thu Oct 31 23:10:13 2013
@@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.util.Pair
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.io.DataInputBuffer;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * HRegion information.
@@ -843,10 +843,10 @@ public class HRegionInfo implements Comp
     builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
     builder.setRegionId(info.getRegionId());
     if (info.getStartKey() != null) {
-      builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
+      builder.setStartKey(ZeroCopyLiteralByteString.wrap(info.getStartKey()));
     }
     if (info.getEndKey() != null) {
-      builder.setEndKey(ByteString.copyFrom(info.getEndKey()));
+      builder.setEndKey(ZeroCopyLiteralByteString.wrap(info.getEndKey()));
     }
     builder.setOffline(info.isOffline());
     builder.setSplit(info.isSplit());

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java Thu Oct 31 23:10:13 2013
@@ -53,8 +53,8 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.WritableComparable;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * HTableDescriptor contains the details about an HBase table  such as the descriptors of
@@ -1435,8 +1435,8 @@ public class HTableDescriptor implements
     builder.setTableName(ProtobufUtil.toProtoTableName(getTableName()));
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
       BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
-      aBuilder.setFirst(ByteString.copyFrom(e.getKey().get()));
-      aBuilder.setSecond(ByteString.copyFrom(e.getValue().get()));
+      aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get()));
+      aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get()));
       builder.addAttributes(aBuilder.build());
     }
     for (HColumnDescriptor hcd: getColumnFamilies()) {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTransition.java Thu Oct 31 23:10:13 2013
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hbase;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.EventType;
@@ -103,10 +104,10 @@ public class RegionTransition {
       org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
         setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
     ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
-      setEventTypeCode(type.getCode()).setRegionName(ByteString.copyFrom(regionName)).
+      setEventTypeCode(type.getCode()).setRegionName(ZeroCopyLiteralByteString.wrap(regionName)).
         setServerName(pbsn);
     builder.setCreateTime(System.currentTimeMillis());
-    if (payload != null) builder.setPayload(ByteString.copyFrom(payload));
+    if (payload != null) builder.setPayload(ZeroCopyLiteralByteString.wrap(payload));
     return new RegionTransition(builder.build());
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java Thu Oct 31 23:10:13 2013
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -29,7 +27,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -133,13 +130,13 @@ public class ClientSmallScanner extends 
           || checkScanStopRow(endKey) || done) {
         close();
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Finished with small scan at " + this.currentRegion);
+          LOG.debug("Finished scan of " + this.currentRegion);
         }
         return false;
       }
       localStartKey = endKey;
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Finished with region " + this.currentRegion);
+        LOG.debug("Finished with " + this.currentRegion);
       }
     } else if (this.lastResult != null) {
       localStartKey = this.lastResult.getRow();
@@ -150,8 +147,7 @@ public class ClientSmallScanner extends 
     }
 
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Advancing internal small scanner to startKey at '"
-          + Bytes.toStringBinary(localStartKey) + "'");
+      LOG.trace("Advancing to startKey at '" + Bytes.toStringBinary(localStartKey) + "'");
     }
     smallScanCallable = getSmallScanCallable(localStartKey, cacheNum);
     if (this.scanMetrics != null && skipRowOfFirstResult == null) {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Thu Oct 31 23:10:13 2013
@@ -1206,9 +1206,9 @@ public class HConnectionManager {
 
           ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
           if (serverName == null) {
-            throw new NoServerForRegionException("No server address listed " +
-              "in " + parentTable + " for region " +
-              regionInfo.getRegionNameAsString() + " containing row " +
+            throw new NoServerForRegionException("No serverName " +
+              "in " + parentTable + " for " +
+              regionInfo.getRegionNameAsString() + " containing " +
               Bytes.toStringBinary(row));
           }
 
@@ -1234,12 +1234,10 @@ public class HConnectionManager {
           }
           if (tries < numTries - 1) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug("locateRegionInMeta parentTable=" +
-                parentTable + ", metaLocation=" +
-                ((metaLocation == null)? "null": "{" + metaLocation + "}") +
-                ", attempt=" + tries + " of " +
-                this.numTries + " failed; retrying after sleep of " +
-                ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage());
+              LOG.debug("locateRegionInMeta failed; parentTable=" + parentTable +
+                ", metaLocation=" + ((metaLocation == null)? "null": "{" + metaLocation + "}") +
+                ", attempt=" + tries + "/" + this.numTries + "; retrying after=" +
+                ConnectionUtils.getPauseTime(this.pause, tries) + "ms; because: " + e.getMessage());
             }
           } else {
             throw e;

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Thu Oct 31 23:10:13 2013
@@ -911,6 +911,7 @@ public class HTable implements HTableInt
    */
   private void backgroundFlushCommits(boolean synchronous) throws
       InterruptedIOException, RetriesExhaustedWithDetailsException {
+    if (this.writeAsyncBuffer.isEmpty()) return;
 
     try {
       do {
@@ -1229,7 +1230,7 @@ public class HTable implements HTableInt
       return;
     }
     flushCommits();
-    if (cleanupPoolOnClose) {
+    if (cleanupPoolOnClose && this.pool != null) {
       this.pool.shutdown();
     }
     if (cleanupConnectionOnClose) {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java Thu Oct 31 23:10:13 2013
@@ -80,13 +80,14 @@ class MultiServerCallable<R> extends Reg
       } else {
         regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions);
       }
-      multiRequestBuilder.addRegionAction(regionActionBuilder.build());
+      RegionAction ra = regionActionBuilder.build();
+      multiRequestBuilder.addRegionAction(ra);
     }
     // Controller optionally carries cell data over the proxy/service boundary and also
     // optionally ferries cell response data back out again.
     PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
     controller.setPriority(getTableName());
-    ClientProtos.MultiResponse responseProto;
+    ClientProtos.MultiResponse responseProto = null;
     ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
     try {
       responseProto = getStub().multi(controller, requestProto);

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java Thu Oct 31 23:10:13 2013
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * ColumnInterpreter for doing Aggregation's with BigDecimal columns. This class
@@ -121,7 +121,7 @@ public class BigDecimalColumnInterpreter
 
   private BigDecimalMsg getProtoForType(BigDecimal t) {
     BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder();
-    return builder.setBigdecimalMsg(ByteString.copyFrom(Bytes.toBytes(t))).build();
+    return builder.setBigdecimalMsg(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(t))).build();
   }
   
   @Override

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java Thu Oct 31 23:10:13 2013
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HC
 import static org.apache.hadoop.hbase.HConstants.LAST_ROW;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
@@ -136,8 +137,8 @@ public class SecureBulkLoadClient {
       if(userToken != null) {
         protoDT =
             SecureBulkLoadProtos.DelegationToken.newBuilder()
-              .setIdentifier(ByteString.copyFrom(userToken.getIdentifier()))
-              .setPassword(ByteString.copyFrom(userToken.getPassword()))
+              .setIdentifier(ZeroCopyLiteralByteString.wrap(userToken.getIdentifier()))
+              .setPassword(ZeroCopyLiteralByteString.wrap(userToken.getPassword()))
               .setKind(userToken.getKind().toString())
               .setService(userToken.getService().toString()).build();
       }
@@ -146,7 +147,7 @@ public class SecureBulkLoadClient {
           new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>();
       for(Pair<byte[], String> el: familyPaths) {
         protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder()
-          .setFamily(ByteString.copyFrom(el.getFirst()))
+          .setFamily(ZeroCopyLiteralByteString.wrap(el.getFirst()))
           .setPath(el.getSecond()).build());
       }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java Thu Oct 31 23:10:13 2013
@@ -18,13 +18,14 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.ByteString;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 
 /** Base class for byte array comparators */
 @InterfaceAudience.Public
@@ -53,7 +54,7 @@ public abstract class ByteArrayComparabl
   ComparatorProtos.ByteArrayComparable convert() {
     ComparatorProtos.ByteArrayComparable.Builder builder =
       ComparatorProtos.ByteArrayComparable.newBuilder();
-    if (value != null) builder.setValue(ByteString.copyFrom(value));
+    if (value != null) builder.setValue(ZeroCopyLiteralByteString.wrap(value));
     return builder.build();
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Thu Oct 31 23:10:13 2013
@@ -18,9 +18,8 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -29,7 +28,9 @@ import org.apache.hadoop.hbase.exception
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
@@ -174,7 +175,7 @@ public class ColumnPaginationFilter exte
       builder.setOffset(this.offset);
     }
     if (this.columnOffset != null) {
-      builder.setColumnOffset(ByteString.copyFrom(this.columnOffset));
+      builder.setColumnOffset(ZeroCopyLiteralByteString.wrap(this.columnOffset));
     }
     return builder.build().toByteArray();
   }

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java Thu Oct 31 23:10:13 2013
@@ -19,9 +19,8 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -30,7 +29,9 @@ import org.apache.hadoop.hbase.exception
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * This filter is used for selecting only those keys with columns that matches
@@ -94,7 +95,7 @@ public class ColumnPrefixFilter extends 
   public byte [] toByteArray() {
     FilterProtos.ColumnPrefixFilter.Builder builder =
       FilterProtos.ColumnPrefixFilter.newBuilder();
-    if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
+    if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix));
     return builder.build().toByteArray();
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java Thu Oct 31 23:10:13 2013
@@ -22,8 +22,9 @@ package org.apache.hadoop.hbase.filter;
 import static org.apache.hadoop.hbase.util.Bytes.len;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -172,9 +173,9 @@ public class ColumnRangeFilter extends F
   public byte [] toByteArray() {
     FilterProtos.ColumnRangeFilter.Builder builder =
       FilterProtos.ColumnRangeFilter.newBuilder();
-    if (this.minColumn != null) builder.setMinColumn(ByteString.copyFrom(this.minColumn));
+    if (this.minColumn != null) builder.setMinColumn(ZeroCopyLiteralByteString.wrap(this.minColumn));
     builder.setMinColumnInclusive(this.minColumnInclusive);
-    if (this.maxColumn != null) builder.setMaxColumn(ByteString.copyFrom(this.maxColumn));
+    if (this.maxColumn != null) builder.setMaxColumn(ZeroCopyLiteralByteString.wrap(this.maxColumn));
     builder.setMaxColumnInclusive(this.maxColumnInclusive);
     return builder.build().toByteArray();
   }

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Thu Oct 31 23:10:13 2013
@@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * A filter for adding inter-column timestamp matching
@@ -225,10 +225,10 @@ public class DependentColumnFilter exten
       FilterProtos.DependentColumnFilter.newBuilder();
     builder.setCompareFilter(super.convert());
     if (this.columnFamily != null) {
-      builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
+      builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily));
     }
     if (this.columnQualifier != null) {
-      builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
+      builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier));
     }
     builder.setDropDependentColumn(this.dropDependentColumn);
     return builder.build().toByteArray();

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java Thu Oct 31 23:10:13 2013
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.filter;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -87,7 +89,7 @@ public class FirstKeyValueMatchingQualif
     FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
       FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
     for (byte[] qualifier : qualifiers) {
-      if (qualifier != null) builder.addQualifiers(ByteString.copyFrom(qualifier));
+      if (qualifier != null) builder.addQualifiers(ZeroCopyLiteralByteString.wrap(qualifier));
     }
     return builder.build().toByteArray();
   }

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java Thu Oct 31 23:10:13 2013
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -146,8 +147,8 @@ public class FuzzyRowFilter extends Filt
       FilterProtos.FuzzyRowFilter.newBuilder();
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
       BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
-      bbpBuilder.setFirst(ByteString.copyFrom(fuzzyData.getFirst()));
-      bbpBuilder.setSecond(ByteString.copyFrom(fuzzyData.getSecond()));
+      bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(fuzzyData.getFirst()));
+      bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(fuzzyData.getSecond()));
       builder.addFuzzyKeysData(bbpBuilder);
     }
     return builder.build().toByteArray();

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Thu Oct 31 23:10:13 2013
@@ -19,16 +19,17 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * A Filter that stops after the given row.  There is no "RowStopFilter" because
@@ -85,7 +86,7 @@ public class InclusiveStopFilter extends
   public byte [] toByteArray() {
     FilterProtos.InclusiveStopFilter.Builder builder =
       FilterProtos.InclusiveStopFilter.newBuilder();
-    if (this.stopRowKey != null) builder.setStopRowKey(ByteString.copyFrom(this.stopRowKey));
+    if (this.stopRowKey != null) builder.setStopRowKey(ZeroCopyLiteralByteString.wrap(this.stopRowKey));
     return builder.build().toByteArray();
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Thu Oct 31 23:10:13 2013
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.ByteString;
+
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -112,7 +114,7 @@ public class MultipleColumnPrefixFilter 
     FilterProtos.MultipleColumnPrefixFilter.Builder builder =
       FilterProtos.MultipleColumnPrefixFilter.newBuilder();
     for (byte [] element : sortedPrefixes) {
-      if (element != null) builder.addSortedPrefixes(ByteString.copyFrom(element));
+      if (element != null) builder.addSortedPrefixes(ZeroCopyLiteralByteString.wrap(element));
     }
     return builder.build().toByteArray();
   }

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java Thu Oct 31 23:10:13 2013
@@ -20,8 +20,9 @@
 package org.apache.hadoop.hbase.filter;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -90,7 +91,7 @@ public class PrefixFilter extends Filter
   public byte [] toByteArray() {
     FilterProtos.PrefixFilter.Builder builder =
       FilterProtos.PrefixFilter.newBuilder();
-    if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
+    if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix));
     return builder.build().toByteArray();
   }
 

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Thu Oct 31 23:10:13 2013
@@ -19,9 +19,9 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -38,8 +38,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
@@ -306,10 +307,10 @@ public class SingleColumnValueFilter ext
     FilterProtos.SingleColumnValueFilter.Builder builder =
       FilterProtos.SingleColumnValueFilter.newBuilder();
     if (this.columnFamily != null) {
-      builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
+      builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily));
     }
     if (this.columnQualifier != null) {
-      builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
+      builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier));
     }
     HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
     builder.setCompareOp(compareOp);

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java Thu Oct 31 23:10:13 2013
@@ -18,9 +18,8 @@
 
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -30,7 +29,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 
-import java.io.IOException;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@@ -60,7 +61,7 @@ public class MasterCoprocessorRpcChannel
 
     final ClientProtos.CoprocessorServiceCall call =
         ClientProtos.CoprocessorServiceCall.newBuilder()
-            .setRow(ByteString.copyFrom(HConstants.EMPTY_BYTE_ARRAY))
+            .setRow(ZeroCopyLiteralByteString.wrap(HConstants.EMPTY_BYTE_ARRAY))
             .setServiceName(method.getService().getFullName())
             .setMethodName(method.getName())
             .setRequest(request.toByteString()).build();

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java Thu Oct 31 23:10:13 2013
@@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@@ -76,7 +76,7 @@ public class RegionCoprocessorRpcChannel
 
     final ClientProtos.CoprocessorServiceCall call =
         ClientProtos.CoprocessorServiceCall.newBuilder()
-            .setRow(ByteString.copyFrom(row))
+            .setRow(ZeroCopyLiteralByteString.wrap(row))
             .setServiceName(method.getService().getFullName())
             .setMethodName(method.getName())
             .setRequest(request.toByteString()).build();

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Thu Oct 31 23:10:13 2013
@@ -137,6 +137,7 @@ import com.google.protobuf.RpcChannel;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.TextFormat;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * Protobufs utility.
@@ -746,17 +747,17 @@ public final class ProtobufUtil {
       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
         attributeBuilder.setName(attribute.getKey());
-        attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
+        attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
         scanBuilder.addAttribute(attributeBuilder.build());
       }
     }
     byte[] startRow = scan.getStartRow();
     if (startRow != null && startRow.length > 0) {
-      scanBuilder.setStartRow(ByteString.copyFrom(startRow));
+      scanBuilder.setStartRow(ZeroCopyLiteralByteString.wrap(startRow));
     }
     byte[] stopRow = scan.getStopRow();
     if (stopRow != null && stopRow.length > 0) {
-      scanBuilder.setStopRow(ByteString.copyFrom(stopRow));
+      scanBuilder.setStopRow(ZeroCopyLiteralByteString.wrap(stopRow));
     }
     if (scan.hasFilter()) {
       scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
@@ -765,12 +766,12 @@ public final class ProtobufUtil {
       Column.Builder columnBuilder = Column.newBuilder();
       for (Map.Entry<byte[],NavigableSet<byte []>>
           family: scan.getFamilyMap().entrySet()) {
-        columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
+        columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
         NavigableSet<byte []> qualifiers = family.getValue();
         columnBuilder.clearQualifier();
         if (qualifiers != null && qualifiers.size() > 0) {
           for (byte [] qualifier: qualifiers) {
-            columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
+            columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
           }
         }
         scanBuilder.addColumn(columnBuilder.build());
@@ -872,7 +873,7 @@ public final class ProtobufUtil {
       final Get get) throws IOException {
     ClientProtos.Get.Builder builder =
       ClientProtos.Get.newBuilder();
-    builder.setRow(ByteString.copyFrom(get.getRow()));
+    builder.setRow(ZeroCopyLiteralByteString.wrap(get.getRow()));
     builder.setCacheBlocks(get.getCacheBlocks());
     builder.setMaxVersions(get.getMaxVersions());
     if (get.getFilter() != null) {
@@ -891,7 +892,7 @@ public final class ProtobufUtil {
       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
         attributeBuilder.setName(attribute.getKey());
-        attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
+        attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
         builder.addAttribute(attributeBuilder.build());
       }
     }
@@ -900,11 +901,11 @@ public final class ProtobufUtil {
       Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
       for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
         NavigableSet<byte[]> qualifiers = family.getValue();
-        columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
+        columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
         columnBuilder.clearQualifier();
         if (qualifiers != null && qualifiers.size() > 0) {
           for (byte[] qualifier: qualifiers) {
-            columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
+            columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
           }
         }
         builder.addColumn(columnBuilder.build());
@@ -933,7 +934,7 @@ public final class ProtobufUtil {
    */
   public static MutationProto toMutation(final Increment increment) {
     MutationProto.Builder builder = MutationProto.newBuilder();
-    builder.setRow(ByteString.copyFrom(increment.getRow()));
+    builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow()));
     builder.setMutateType(MutationType.INCREMENT);
     builder.setDurability(toDurability(increment.getDurability()));
     TimeRange timeRange = increment.getTimeRange();
@@ -947,14 +948,14 @@ public final class ProtobufUtil {
     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
    for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) {
-      columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
+      columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
       columnBuilder.clearQualifierValue();
       List<Cell> values = family.getValue();
       if (values != null && values.size() > 0) {
         for (Cell cell: values) {
           KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-          valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
-          valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
+          valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(kv.getQualifier()));
+          valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(kv.getValue()));
           columnBuilder.addQualifierValue(valueBuilder.build());
         }
       }
@@ -977,12 +978,12 @@ public final class ProtobufUtil {
     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
     for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
-      columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
+      columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
       columnBuilder.clearQualifierValue();
       for (Cell cell: family.getValue()) {
         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
-        valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
+        valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(kv.getQualifier()));
+        valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(kv.getValue()));
         valueBuilder.setTimestamp(kv.getTimestamp());
         if (type == MutationType.DELETE) {
           KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType());
@@ -1020,7 +1021,7 @@ public final class ProtobufUtil {
   private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
       final Mutation mutation) {
     MutationProto.Builder builder = MutationProto.newBuilder();
-    builder.setRow(ByteString.copyFrom(mutation.getRow()));
+    builder.setRow(ZeroCopyLiteralByteString.wrap(mutation.getRow()));
     builder.setMutateType(type);
     builder.setDurability(toDurability(mutation.getDurability()));
     builder.setTimestamp(mutation.getTimeStamp());
@@ -1029,7 +1030,7 @@ public final class ProtobufUtil {
       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
         attributeBuilder.setName(attribute.getKey());
-        attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
+        attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
         builder.addAttribute(attributeBuilder.build());
       }
     }
@@ -1144,7 +1145,7 @@ public final class ProtobufUtil {
   public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) {
     ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
     builder.setName(comparator.getClass().getName());
-    builder.setSerializedComparator(ByteString.copyFrom(comparator.toByteArray()));
+    builder.setSerializedComparator(ZeroCopyLiteralByteString.wrap(comparator.toByteArray()));
     return builder.build();
   }
 
@@ -1206,7 +1207,7 @@ public final class ProtobufUtil {
   public static FilterProtos.Filter toFilter(Filter filter) throws IOException {
     FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder();
     builder.setName(filter.getClass().getName());
-    builder.setSerializedFilter(ByteString.copyFrom(filter.toByteArray()));
+    builder.setSerializedFilter(ZeroCopyLiteralByteString.wrap(filter.toByteArray()));
     return builder.build();
   }
 
@@ -1691,10 +1692,10 @@ public final class ProtobufUtil {
             AccessControlProtos.TablePermission.newBuilder();
         builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
         if (tablePerm.hasFamily()) {
-          builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
+          builder.setFamily(ZeroCopyLiteralByteString.wrap(tablePerm.getFamily()));
         }
         if (tablePerm.hasQualifier()) {
-          builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
+          builder.setQualifier(ZeroCopyLiteralByteString.wrap(tablePerm.getQualifier()));
         }
         for (Permission.Action a : perm.getActions()) {
           builder.addAction(toPermissionAction(a));
@@ -1783,7 +1784,7 @@ public final class ProtobufUtil {
    */
   public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
     return AccessControlProtos.UserPermission.newBuilder()
-        .setUser(ByteString.copyFrom(perm.getUser()))
+        .setUser(ZeroCopyLiteralByteString.wrap(perm.getUser()))
         .setPermission(toPermission(perm))
         .build();
   }
@@ -2055,8 +2056,8 @@ public final class ProtobufUtil {
    */
   public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
     AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
-    builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
-    builder.setPassword(ByteString.copyFrom(token.getPassword()));
+    builder.setIdentifier(ZeroCopyLiteralByteString.wrap(token.getIdentifier()));
+    builder.setPassword(ZeroCopyLiteralByteString.wrap(token.getPassword()));
     if (token.getService() != null) {
       builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
     }
@@ -2239,9 +2240,9 @@ public final class ProtobufUtil {
     // input / output paths are relative to the store dir
     // store dir is relative to region dir
     CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
-        .setTableName(ByteString.copyFrom(info.getTableName()))
-        .setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
-        .setFamilyName(ByteString.copyFrom(family))
+        .setTableName(ZeroCopyLiteralByteString.wrap(info.getTableName()))
+        .setEncodedRegionName(ZeroCopyLiteralByteString.wrap(info.getEncodedNameAsBytes()))
+        .setFamilyName(ZeroCopyLiteralByteString.wrap(family))
         .setStoreHomeDir(storeDir.getName()); //make relative
     for (Path inputPath : inputPaths) {
       builder.addCompactionInput(inputPath.getName()); //relative path
@@ -2318,8 +2319,8 @@ public final class ProtobufUtil {
 
   public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
     return HBaseProtos.TableName.newBuilder()
-        .setNamespace(ByteString.copyFrom(tableName.getNamespace()))
-        .setQualifier(ByteString.copyFrom(tableName.getQualifier())).build();
+        .setNamespace(ZeroCopyLiteralByteString.wrap(tableName.getNamespace()))
+        .setQualifier(ZeroCopyLiteralByteString.wrap(tableName.getQualifier())).build();
   }
 
   public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Thu Oct 31 23:10:13 2013
@@ -22,15 +22,16 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CellScannable;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Action;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -38,7 +39,6 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
@@ -99,9 +99,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Triple;
-import org.mortbay.log.Log;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * Helper utility to build protocol buffer requests,
@@ -133,10 +133,10 @@ public final class RequestConverter {
     builder.setRegion(region);
 
     Column.Builder columnBuilder = Column.newBuilder();
-    columnBuilder.setFamily(ByteString.copyFrom(family));
+    columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
     ClientProtos.Get.Builder getBuilder =
       ClientProtos.Get.newBuilder();
-    getBuilder.setRow(ByteString.copyFrom(row));
+    getBuilder.setRow(ZeroCopyLiteralByteString.wrap(row));
     getBuilder.addColumn(columnBuilder.build());
     getBuilder.setClosestRowBefore(true);
     builder.setGet(getBuilder.build());
@@ -181,14 +181,14 @@ public final class RequestConverter {
     builder.setRegion(region);
 
     MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
-    mutateBuilder.setRow(ByteString.copyFrom(row));
+    mutateBuilder.setRow(ZeroCopyLiteralByteString.wrap(row));
     mutateBuilder.setMutateType(MutationType.INCREMENT);
     mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
-    columnBuilder.setFamily(ByteString.copyFrom(family));
+    columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
-    valueBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(amount)));
-    valueBuilder.setQualifier(ByteString.copyFrom(qualifier));
+    valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(amount)));
+    valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
     columnBuilder.addQualifierValue(valueBuilder.build());
     mutateBuilder.addColumnValue(columnBuilder.build());
     builder.setMutation(mutateBuilder.build());
@@ -466,7 +466,7 @@ public final class RequestConverter {
     builder.setRegion(region);
     FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder();
     for (Pair<byte[], String> familyPath: familyPaths) {
-      familyPathBuilder.setFamily(ByteString.copyFrom(familyPath.getFirst()));
+      familyPathBuilder.setFamily(ZeroCopyLiteralByteString.wrap(familyPath.getFirst()));
       familyPathBuilder.setPath(familyPath.getSecond());
       builder.addFamilyPath(familyPathBuilder.build());
     }
@@ -629,7 +629,7 @@ public final class RequestConverter {
    RegionSpecifier region = buildRegionSpecifier(
      RegionSpecifierType.REGION_NAME, regionName);
    builder.setRegion(region);
-   builder.addFamily(ByteString.copyFrom(family));
+   builder.addFamily(ZeroCopyLiteralByteString.wrap(family));
    return builder.build();
  }
 
@@ -775,7 +775,7 @@ public final class RequestConverter {
      RegionSpecifierType.REGION_NAME, regionName);
    builder.setRegion(region);
    if (splitPoint != null) {
-     builder.setSplitPoint(ByteString.copyFrom(splitPoint));
+     builder.setSplitPoint(ZeroCopyLiteralByteString.wrap(splitPoint));
    }
    return builder.build();
  }
@@ -815,7 +815,7 @@ public final class RequestConverter {
    builder.setRegion(region);
    builder.setMajor(major);
    if (family != null) {
-     builder.setFamily(ByteString.copyFrom(family));
+     builder.setFamily(ZeroCopyLiteralByteString.wrap(family));
    }
    return builder.build();
  }
@@ -874,7 +874,7 @@ public final class RequestConverter {
   public static RegionSpecifier buildRegionSpecifier(
       final RegionSpecifierType type, final byte[] value) {
     RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder();
-    regionBuilder.setValue(ByteString.copyFrom(value));
+    regionBuilder.setValue(ZeroCopyLiteralByteString.wrap(value));
     regionBuilder.setType(type);
     return regionBuilder.build();
   }
@@ -895,9 +895,9 @@ public final class RequestConverter {
       final ByteArrayComparable comparator,
       final CompareType compareType) throws IOException {
     Condition.Builder builder = Condition.newBuilder();
-    builder.setRow(ByteString.copyFrom(row));
-    builder.setFamily(ByteString.copyFrom(family));
-    builder.setQualifier(ByteString.copyFrom(qualifier));
+    builder.setRow(ZeroCopyLiteralByteString.wrap(row));
+    builder.setFamily(ZeroCopyLiteralByteString.wrap(family));
+    builder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
     builder.setComparator(ProtobufUtil.toComparator(comparator));
     builder.setCompareType(compareType);
     return builder.build();
@@ -929,7 +929,7 @@ public final class RequestConverter {
       final TableName tableName, final byte [] columnName) {
     DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
     builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
-    builder.setColumnName(ByteString.copyFrom(columnName));
+    builder.setColumnName(ZeroCopyLiteralByteString.wrap(columnName));
     return builder.build();
   }
 
@@ -1069,7 +1069,7 @@ public final class RequestConverter {
     builder.setTableSchema(hTableDesc.convert());
     if (splitKeys != null) {
       for (byte [] splitKey : splitKeys) {
-        builder.addSplitKeys(ByteString.copyFrom(splitKey));
+        builder.addSplitKeys(ZeroCopyLiteralByteString.wrap(splitKey));
       }
     }
     return builder.build();
@@ -1222,7 +1222,7 @@ public final class RequestConverter {
   public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest(
       byte[] regionName) {
     return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName(
-        ByteString.copyFrom(regionName)).build();
+        ZeroCopyLiteralByteString.wrap(regionName)).build();
   }
 
   /**
@@ -1277,10 +1277,10 @@ public final class RequestConverter {
     permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
 
     if (family != null) {
-      permissionBuilder.setFamily(ByteString.copyFrom(family));
+      permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
     }
     if (qualifier != null) {
-      permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
+      permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
     }
     ret.setType(AccessControlProtos.Permission.Type.Table)
        .setTablePermission(permissionBuilder);
@@ -1373,10 +1373,10 @@ public final class RequestConverter {
       permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
     }
     if (family != null) {
-      permissionBuilder.setFamily(ByteString.copyFrom(family));
+      permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
     }
     if (qualifier != null) {
-      permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
+      permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
     }
     ret.setType(AccessControlProtos.Permission.Type.Table)
        .setTablePermission(permissionBuilder);

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Thu Oct 31 23:10:13 2013
@@ -69,8 +69,8 @@ import org.apache.zookeeper.proto.Delete
 import org.apache.zookeeper.proto.SetDataRequest;
 import org.apache.zookeeper.server.ZooKeeperSaslServer;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * Internal HBase utility class for ZooKeeper.
@@ -1940,7 +1940,7 @@ public class ZKUtil {
     if (storeSequenceIds != null) {
       for (byte[] columnFamilyName : storeSequenceIds.keySet()) {
         Long curSeqId = storeSequenceIds.get(columnFamilyName);
-        storeSequenceIdBuilder.setFamilyName(ByteString.copyFrom(columnFamilyName));
+        storeSequenceIdBuilder.setFamilyName(ZeroCopyLiteralByteString.wrap(columnFamilyName));
         storeSequenceIdBuilder.setSequenceId(curSeqId);
         regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
         storeSequenceIdBuilder.clear();

Added: hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java?rev=1537756&view=auto
==============================================================================
--- hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java (added)
+++ hbase/branches/0.96/hbase-protocol/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java Thu Oct 31 23:10:13 2013
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.google.protobuf;  // This is a lie.
+
+/**
+ * Helper class to extract byte arrays from {@link ByteString} without copy.
+ * <p>
+ * Without this protobufs would force us to copy every single byte array out
+ * of the objects de-serialized from the wire (which already do one copy, on
+ * top of the copies the JVM does to go from kernel buffer to C buffer and
+ * from C buffer to JVM buffer).
+ *
+ * @since 0.96.1
+ */
+public final class ZeroCopyLiteralByteString extends LiteralByteString {
+  // Gotten from AsyncHBase code base with permission.
+  /** Private constructor so this class cannot be instantiated. */
+  private ZeroCopyLiteralByteString() {
+    super(null);
+    throw new UnsupportedOperationException("Should never be here.");
+  }
+
+  /**
+   * Wraps a byte array in a {@link ByteString} without copying it.
+   */
+  public static ByteString wrap(final byte[] array) {
+    return new LiteralByteString(array);
+  }
+
+  // TODO:
+  // ZeroCopyLiteralByteString.wrap(this.buf, 0, this.count);
+
+  /**
+   * Extracts the byte array from the given {@link ByteString} without copy.
+   * @param buf A buffer from which to extract the array.  This buffer must be
+   * actually an instance of a {@code LiteralByteString}.
+   */
+  public static byte[] zeroCopyGetBytes(final LiteralByteString buf) {
+    return buf.bytes;
+  }
+}

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java Thu Oct 31 23:10:13 2013
@@ -557,4 +557,4 @@ public class MetaEditor {
         Bytes.toBytes(openSeqNum));
     return p;
   }
-}
+}
\ No newline at end of file

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java Thu Oct 31 23:10:13 2013
@@ -24,13 +24,9 @@ import java.io.OutputStream;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.codec.BaseDecoder;
-import org.apache.hadoop.hbase.codec.BaseEncoder;
-import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
 
 import com.google.protobuf.ByteString;
-import org.apache.hadoop.classification.InterfaceStability;
 
 /**
  * Codec that just writes out Cell as a protobuf Cell Message.  Does not write the mvcc stamp.

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java Thu Oct 31 23:10:13 2013
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * A reference to the top or bottom half of a store file where 'bottom' is the first half
@@ -194,7 +195,7 @@ public class Reference {
     FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
     builder.setRange(isTopFileRegion(getFileRegion())?
       FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
-    builder.setSplitkey(ByteString.copyFrom(getSplitKey()));
+    builder.setSplitkey(ZeroCopyLiteralByteString.wrap(getSplitKey()));
     return builder.build();
   }
 

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Thu Oct 31 23:10:13 2013
@@ -55,23 +55,22 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
-import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * File format for hbase.
@@ -785,8 +784,8 @@ public class HFile {
       HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
       for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
         HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
-        bbpBuilder.setFirst(ByteString.copyFrom(e.getKey()));
-        bbpBuilder.setSecond(ByteString.copyFrom(e.getValue()));
+        bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey()));
+        bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue()));
         builder.addMapEntry(bbpBuilder.build());
       }
       out.write(ProtobufUtil.PB_MAGIC);

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java Thu Oct 31 23:10:13 2013
@@ -26,7 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -42,11 +41,10 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 
-import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 @InterfaceAudience.Private
 public class ReplicationProtbufUtil {
@@ -91,8 +89,8 @@ public class ReplicationProtbufUtil {
       WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
       HLogKey key = entry.getKey();
       keyBuilder.setEncodedRegionName(
-        ByteString.copyFrom(key.getEncodedRegionName()));
-      keyBuilder.setTableName(ByteString.copyFrom(key.getTablename().getName()));
+        ZeroCopyLiteralByteString.wrap(key.getEncodedRegionName()));
+      keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName()));
       keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
       keyBuilder.setWriteTime(key.getWriteTime());
       for(UUID clusterId : key.getClusterIds()) {
@@ -104,7 +102,7 @@ public class ReplicationProtbufUtil {
       NavigableMap<byte[], Integer> scopes = key.getScopes();
       if (scopes != null && !scopes.isEmpty()) {
         for (Map.Entry<byte[], Integer> scope: scopes.entrySet()) {
-          scopeBuilder.setFamily(ByteString.copyFrom(scope.getKey()));
+          scopeBuilder.setFamily(ZeroCopyLiteralByteString.wrap(scope.getKey()));
           WALProtos.ScopeType scopeType =
               WALProtos.ScopeType.valueOf(scope.getValue().intValue());
           scopeBuilder.setScopeType(scopeType);

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Oct 31 23:10:13 2013
@@ -195,7 +195,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -233,6 +232,7 @@ import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.TextFormat;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * HRegionServer makes a set of HRegions available to clients. It checks in with
@@ -1275,7 +1275,7 @@ public class HRegionServer implements Cl
     RegionLoad.Builder regionLoad = RegionLoad.newBuilder();
     RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
-    regionSpecifier.setValue(ByteString.copyFrom(name));
+    regionSpecifier.setValue(ZeroCopyLiteralByteString.wrap(name));
     regionLoad.setRegionSpecifier(regionSpecifier.build())
       .setStores(stores)
       .setStorefiles(storefiles)
@@ -3898,7 +3898,7 @@ public class HRegionServer implements Cl
       RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
       if (regionsToFlush != null) {
         for (byte[] region: regionsToFlush) {
-          builder.addRegionToFlush(ByteString.copyFrom(region));
+          builder.addRegionToFlush(ZeroCopyLiteralByteString.wrap(region));
         }
       }
       return builder.build();

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java Thu Oct 31 23:10:13 2013
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -35,9 +34,9 @@ import java.util.UUID;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
@@ -47,6 +46,7 @@ import org.apache.hadoop.io.WritableComp
 import org.apache.hadoop.io.WritableUtils;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.ZeroCopyLiteralByteString;
 
 /**
  * A Key for an entry in the change log.
@@ -425,8 +425,8 @@ public class HLogKey implements Writable
       WALCellCodec.ByteStringCompressor compressor) throws IOException {
     WALKey.Builder builder = WALKey.newBuilder();
     if (compressionContext == null) {
-      builder.setEncodedRegionName(ByteString.copyFrom(this.encodedRegionName));
-      builder.setTableName(ByteString.copyFrom(this.tablename.getName()));
+      builder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(this.encodedRegionName));
+      builder.setTableName(ZeroCopyLiteralByteString.wrap(this.tablename.getName()));
     } else {
       builder.setEncodedRegionName(
           compressor.compress(this.encodedRegionName, compressionContext.regionDict));
@@ -443,7 +443,7 @@ public class HLogKey implements Writable
     }
     if (scopes != null) {
       for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
-        ByteString family = (compressionContext == null) ? ByteString.copyFrom(e.getKey())
+        ByteString family = (compressionContext == null) ? ZeroCopyLiteralByteString.wrap(e.getKey())
             : compressor.compress(e.getKey(), compressionContext.familyDict);
         builder.addScopes(FamilyScope.newBuilder()
             .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java?rev=1537756&r1=1537755&r2=1537756&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/CellModel.java Thu Oct 31 23:10:13 2013
@@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
 import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell;
-
-import com.google.protobuf.ByteString;
 import org.codehaus.jackson.annotate.JsonProperty;
 
+import com.google.protobuf.ZeroCopyLiteralByteString;
+
 /**
  * Representation of a cell. A cell is a single value associated a column and
  * optional qualifier, and either the timestamp when it was stored or the user-
@@ -185,8 +185,8 @@ public class CellModel implements Protob
   @Override
   public byte[] createProtobufOutput() {
     Cell.Builder builder = Cell.newBuilder();
-    builder.setColumn(ByteString.copyFrom(getColumn()));
-    builder.setData(ByteString.copyFrom(getValue()));
+    builder.setColumn(ZeroCopyLiteralByteString.wrap(getColumn()));
+    builder.setData(ZeroCopyLiteralByteString.wrap(getValue()));
     if (hasUserTimestamp()) {
       builder.setTimestamp(getTimestamp());
     }