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());
}