HBASE-10431 Rename com.google.protobuf.ZeroCopyLiteralByteString

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1562576 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
nkeywal 2014-01-29 20:31:31 +00:00
parent 7756ace9a2
commit e0c1033891
49 changed files with 281 additions and 229 deletions

View File

@ -26,6 +26,7 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
@ -40,8 +41,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Re
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.VersionedWritable; import org.apache.hadoop.io.VersionedWritable;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Status information on the HBase cluster. * Status information on the HBase cluster.
@ -335,7 +334,7 @@ public class ClusterStatus extends VersionedWritable {
ClusterStatusProtos.RegionState rs = rit.getValue().convert(); ClusterStatusProtos.RegionState rs = rit.getValue().convert();
RegionSpecifier.Builder spec = RegionSpecifier.Builder spec =
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME); RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
spec.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(rit.getKey()))); spec.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(rit.getKey())));
RegionInTransition pbRIT = RegionInTransition pbRIT =
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build(); RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();

View File

@ -19,8 +19,8 @@
package org.apache.hadoop.hbase; package org.apache.hadoop.hbase;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -1225,11 +1225,11 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
*/ */
public ColumnFamilySchema convert() { public ColumnFamilySchema convert() {
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder(); ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
builder.setName(ZeroCopyLiteralByteString.wrap(getName())); builder.setName(HBaseZeroCopyByteString.wrap(getName()));
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) { for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder(); BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get())); aBuilder.setFirst(HBaseZeroCopyByteString.wrap(e.getKey().get()));
aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get())); aBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build()); builder.addAttributes(aBuilder.build());
} }
for (Map.Entry<String, String> e : this.configuration.entrySet()) { for (Map.Entry<String, String> e : this.configuration.entrySet()) {

View File

@ -29,6 +29,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -47,7 +48,6 @@ import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataInputBuffer;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* HRegion information. * HRegion information.
@ -842,10 +842,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable())); builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
builder.setRegionId(info.getRegionId()); builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) { if (info.getStartKey() != null) {
builder.setStartKey(ZeroCopyLiteralByteString.wrap(info.getStartKey())); builder.setStartKey(HBaseZeroCopyByteString.wrap(info.getStartKey()));
} }
if (info.getEndKey() != null) { if (info.getEndKey() != null) {
builder.setEndKey(ZeroCopyLiteralByteString.wrap(info.getEndKey())); builder.setEndKey(HBaseZeroCopyByteString.wrap(info.getEndKey()));
} }
builder.setOffline(info.isOffline()); builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit()); builder.setSplit(info.isSplit());

View File

@ -34,6 +34,7 @@ import java.util.TreeMap;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -54,7 +55,6 @@ import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparable;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* HTableDescriptor contains the details about an HBase table such as the descriptors of * HTableDescriptor contains the details about an HBase table such as the descriptors of
@ -1430,8 +1430,8 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
builder.setTableName(ProtobufUtil.toProtoTableName(getTableName())); builder.setTableName(ProtobufUtil.toProtoTableName(getTableName()));
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) { for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder(); BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get())); aBuilder.setFirst(HBaseZeroCopyByteString.wrap(e.getKey().get()));
aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get())); aBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build()); builder.addAttributes(aBuilder.build());
} }
for (HColumnDescriptor hcd: getColumnFamilies()) { for (HColumnDescriptor hcd: getColumnFamilies()) {

View File

@ -17,8 +17,8 @@
*/ */
package org.apache.hadoop.hbase; package org.apache.hadoop.hbase;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -104,10 +104,10 @@ public class RegionTransition {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(). org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build(); setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder(). ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
setEventTypeCode(type.getCode()).setRegionName(ZeroCopyLiteralByteString.wrap(regionName)). setEventTypeCode(type.getCode()).setRegionName(HBaseZeroCopyByteString.wrap(regionName)).
setServerName(pbsn); setServerName(pbsn);
builder.setCreateTime(System.currentTimeMillis()); builder.setCreateTime(System.currentTimeMillis());
if (payload != null) builder.setPayload(ZeroCopyLiteralByteString.wrap(payload)); if (payload != null) builder.setPayload(HBaseZeroCopyByteString.wrap(payload));
return new RegionTransition(builder.build()); return new RegionTransition(builder.build());
} }

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.math.BigDecimal; import java.math.BigDecimal;
import java.math.RoundingMode; import java.math.RoundingMode;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
@ -30,8 +31,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* ColumnInterpreter for doing Aggregation's with BigDecimal columns. This class * ColumnInterpreter for doing Aggregation's with BigDecimal columns. This class
* is required at the RegionServer also. * is required at the RegionServer also.
@ -122,7 +121,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
private BigDecimalMsg getProtoForType(BigDecimal t) { private BigDecimalMsg getProtoForType(BigDecimal t) {
BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder(); BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder();
return builder.setBigdecimalMsg(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(t))).build(); return builder.setBigdecimalMsg(HBaseZeroCopyByteString.wrap(Bytes.toBytes(t))).build();
} }
@Override @Override

View File

@ -21,8 +21,7 @@ package org.apache.hadoop.hbase.client.coprocessor;
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW; import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.HConstants.LAST_ROW; import static org.apache.hadoop.hbase.HConstants.LAST_ROW;
import com.google.protobuf.ByteString; import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -137,8 +136,8 @@ public class SecureBulkLoadClient {
if(userToken != null) { if(userToken != null) {
protoDT = protoDT =
SecureBulkLoadProtos.DelegationToken.newBuilder() SecureBulkLoadProtos.DelegationToken.newBuilder()
.setIdentifier(ZeroCopyLiteralByteString.wrap(userToken.getIdentifier())) .setIdentifier(HBaseZeroCopyByteString.wrap(userToken.getIdentifier()))
.setPassword(ZeroCopyLiteralByteString.wrap(userToken.getPassword())) .setPassword(HBaseZeroCopyByteString.wrap(userToken.getPassword()))
.setKind(userToken.getKind().toString()) .setKind(userToken.getKind().toString())
.setService(userToken.getService().toString()).build(); .setService(userToken.getService().toString()).build();
} }
@ -147,7 +146,7 @@ public class SecureBulkLoadClient {
new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>(); new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>();
for(Pair<byte[], String> el: familyPaths) { for(Pair<byte[], String> el: familyPaths) {
protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder() protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder()
.setFamily(ZeroCopyLiteralByteString.wrap(el.getFirst())) .setFamily(HBaseZeroCopyByteString.wrap(el.getFirst()))
.setPath(el.getSecond()).build()); .setPath(el.getSecond()).build());
} }

View File

@ -18,14 +18,13 @@
*/ */
package org.apache.hadoop.hbase.filter; package org.apache.hadoop.hbase.filter;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** Base class for byte array comparators */ /** Base class for byte array comparators */
@InterfaceAudience.Public @InterfaceAudience.Public
@ -54,7 +53,7 @@ public abstract class ByteArrayComparable implements Comparable<byte[]> {
ComparatorProtos.ByteArrayComparable convert() { ComparatorProtos.ByteArrayComparable convert() {
ComparatorProtos.ByteArrayComparable.Builder builder = ComparatorProtos.ByteArrayComparable.Builder builder =
ComparatorProtos.ByteArrayComparable.newBuilder(); ComparatorProtos.ByteArrayComparable.newBuilder();
if (value != null) builder.setValue(ZeroCopyLiteralByteString.wrap(value)); if (value != null) builder.setValue(HBaseZeroCopyByteString.wrap(value));
return builder.build(); return builder.build();
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList; import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset. * A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
@ -175,7 +175,7 @@ public class ColumnPaginationFilter extends FilterBase
builder.setOffset(this.offset); builder.setOffset(this.offset);
} }
if (this.columnOffset != null) { if (this.columnOffset != null) {
builder.setColumnOffset(ZeroCopyLiteralByteString.wrap(this.columnOffset)); builder.setColumnOffset(HBaseZeroCopyByteString.wrap(this.columnOffset));
} }
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList; import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -31,7 +32,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* This filter is used for selecting only those keys with columns that matches * This filter is used for selecting only those keys with columns that matches
@ -95,7 +95,7 @@ public class ColumnPrefixFilter extends FilterBase {
public byte [] toByteArray() { public byte [] toByteArray() {
FilterProtos.ColumnPrefixFilter.Builder builder = FilterProtos.ColumnPrefixFilter.Builder builder =
FilterProtos.ColumnPrefixFilter.newBuilder(); FilterProtos.ColumnPrefixFilter.newBuilder();
if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix)); if (this.prefix != null) builder.setPrefix(HBaseZeroCopyByteString.wrap(this.prefix));
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.filter;
import static org.apache.hadoop.hbase.util.Bytes.len; import static org.apache.hadoop.hbase.util.Bytes.len;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -173,9 +173,9 @@ public class ColumnRangeFilter extends FilterBase {
public byte [] toByteArray() { public byte [] toByteArray() {
FilterProtos.ColumnRangeFilter.Builder builder = FilterProtos.ColumnRangeFilter.Builder builder =
FilterProtos.ColumnRangeFilter.newBuilder(); FilterProtos.ColumnRangeFilter.newBuilder();
if (this.minColumn != null) builder.setMinColumn(ZeroCopyLiteralByteString.wrap(this.minColumn)); if (this.minColumn != null) builder.setMinColumn(HBaseZeroCopyByteString.wrap(this.minColumn));
builder.setMinColumnInclusive(this.minColumnInclusive); builder.setMinColumnInclusive(this.minColumnInclusive);
if (this.maxColumn != null) builder.setMaxColumn(ZeroCopyLiteralByteString.wrap(this.maxColumn)); if (this.maxColumn != null) builder.setMaxColumn(HBaseZeroCopyByteString.wrap(this.maxColumn));
builder.setMaxColumnInclusive(this.maxColumnInclusive); builder.setMaxColumnInclusive(this.maxColumnInclusive);
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -25,6 +25,7 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* A filter for adding inter-column timestamp matching * A filter for adding inter-column timestamp matching
@ -225,10 +225,10 @@ public class DependentColumnFilter extends CompareFilter {
FilterProtos.DependentColumnFilter.newBuilder(); FilterProtos.DependentColumnFilter.newBuilder();
builder.setCompareFilter(super.convert()); builder.setCompareFilter(super.convert());
if (this.columnFamily != null) { if (this.columnFamily != null) {
builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily)); builder.setColumnFamily(HBaseZeroCopyByteString.wrap(this.columnFamily));
} }
if (this.columnQualifier != null) { if (this.columnQualifier != null) {
builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier)); builder.setColumnQualifier(HBaseZeroCopyByteString.wrap(this.columnQualifier));
} }
builder.setDropDependentColumn(this.dropDependentColumn); builder.setDropDependentColumn(this.dropDependentColumn);
return builder.build().toByteArray(); return builder.build().toByteArray();

View File

@ -19,8 +19,8 @@
package org.apache.hadoop.hbase.filter; package org.apache.hadoop.hbase.filter;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -89,7 +89,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder = FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder(); FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
for (byte[] qualifier : qualifiers) { for (byte[] qualifier : qualifiers) {
if (qualifier != null) builder.addQualifiers(ZeroCopyLiteralByteString.wrap(qualifier)); if (qualifier != null) builder.addQualifiers(HBaseZeroCopyByteString.wrap(qualifier));
} }
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -17,8 +17,8 @@
*/ */
package org.apache.hadoop.hbase.filter; package org.apache.hadoop.hbase.filter;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -147,8 +147,8 @@ public class FuzzyRowFilter extends FilterBase {
FilterProtos.FuzzyRowFilter.newBuilder(); FilterProtos.FuzzyRowFilter.newBuilder();
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) { for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder(); BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(fuzzyData.getFirst())); bbpBuilder.setFirst(HBaseZeroCopyByteString.wrap(fuzzyData.getFirst()));
bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(fuzzyData.getSecond())); bbpBuilder.setSecond(HBaseZeroCopyByteString.wrap(fuzzyData.getSecond()));
builder.addFuzzyKeysData(bbpBuilder); builder.addFuzzyKeysData(bbpBuilder);
} }
return builder.build().toByteArray(); return builder.build().toByteArray();

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList; import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -29,7 +30,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* A Filter that stops after the given row. There is no "RowStopFilter" because * A Filter that stops after the given row. There is no "RowStopFilter" because
@ -86,7 +86,7 @@ public class InclusiveStopFilter extends FilterBase {
public byte [] toByteArray() { public byte [] toByteArray() {
FilterProtos.InclusiveStopFilter.Builder builder = FilterProtos.InclusiveStopFilter.Builder builder =
FilterProtos.InclusiveStopFilter.newBuilder(); FilterProtos.InclusiveStopFilter.newBuilder();
if (this.stopRowKey != null) builder.setStopRowKey(ZeroCopyLiteralByteString.wrap(this.stopRowKey)); if (this.stopRowKey != null) builder.setStopRowKey(HBaseZeroCopyByteString.wrap(this.stopRowKey));
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.filter; package org.apache.hadoop.hbase.filter;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -114,7 +114,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
FilterProtos.MultipleColumnPrefixFilter.Builder builder = FilterProtos.MultipleColumnPrefixFilter.Builder builder =
FilterProtos.MultipleColumnPrefixFilter.newBuilder(); FilterProtos.MultipleColumnPrefixFilter.newBuilder();
for (byte [] element : sortedPrefixes) { for (byte [] element : sortedPrefixes) {
if (element != null) builder.addSortedPrefixes(ZeroCopyLiteralByteString.wrap(element)); if (element != null) builder.addSortedPrefixes(HBaseZeroCopyByteString.wrap(element));
} }
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -20,8 +20,8 @@
package org.apache.hadoop.hbase.filter; package org.apache.hadoop.hbase.filter;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -91,7 +91,7 @@ public class PrefixFilter extends FilterBase {
public byte [] toByteArray() { public byte [] toByteArray() {
FilterProtos.PrefixFilter.Builder builder = FilterProtos.PrefixFilter.Builder builder =
FilterProtos.PrefixFilter.newBuilder(); FilterProtos.PrefixFilter.newBuilder();
if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix)); if (this.prefix != null) builder.setPrefix(HBaseZeroCopyByteString.wrap(this.prefix));
return builder.build().toByteArray(); return builder.build().toByteArray();
} }

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -40,7 +41,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException; 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} * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
@ -307,10 +307,10 @@ public class SingleColumnValueFilter extends FilterBase {
FilterProtos.SingleColumnValueFilter.Builder builder = FilterProtos.SingleColumnValueFilter.Builder builder =
FilterProtos.SingleColumnValueFilter.newBuilder(); FilterProtos.SingleColumnValueFilter.newBuilder();
if (this.columnFamily != null) { if (this.columnFamily != null) {
builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily)); builder.setColumnFamily(HBaseZeroCopyByteString.wrap(this.columnFamily));
} }
if (this.columnQualifier != null) { if (this.columnQualifier != null) {
builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier)); builder.setColumnQualifier(HBaseZeroCopyByteString.wrap(this.columnQualifier));
} }
HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name()); HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
builder.setCompareOp(compareOp); builder.setCompareOp(compareOp);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException; import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -31,7 +32,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServic
import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors;
import com.google.protobuf.Message; 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 * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@ -61,7 +61,7 @@ public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
final ClientProtos.CoprocessorServiceCall call = final ClientProtos.CoprocessorServiceCall call =
ClientProtos.CoprocessorServiceCall.newBuilder() ClientProtos.CoprocessorServiceCall.newBuilder()
.setRow(ZeroCopyLiteralByteString.wrap(HConstants.EMPTY_BYTE_ARRAY)) .setRow(HBaseZeroCopyByteString.wrap(HConstants.EMPTY_BYTE_ARRAY))
.setServiceName(method.getService().getFullName()) .setServiceName(method.getService().getFullName())
.setMethodName(method.getName()) .setMethodName(method.getName())
.setRequest(request.toByteString()).build(); .setRequest(request.toByteString()).build();

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException; import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.Descriptors; import com.google.protobuf.Descriptors;
import com.google.protobuf.Message; 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 * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
@ -76,7 +76,7 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
final ClientProtos.CoprocessorServiceCall call = final ClientProtos.CoprocessorServiceCall call =
ClientProtos.CoprocessorServiceCall.newBuilder() ClientProtos.CoprocessorServiceCall.newBuilder()
.setRow(ZeroCopyLiteralByteString.wrap(row)) .setRow(HBaseZeroCopyByteString.wrap(row))
.setServiceName(method.getService().getFullName()) .setServiceName(method.getService().getFullName())
.setMethodName(method.getName()) .setMethodName(method.getName())
.setRequest(request.toByteString()).build(); .setRequest(request.toByteString()).build();

View File

@ -36,6 +36,7 @@ import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.NavigableSet; import java.util.NavigableSet;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -141,7 +142,6 @@ import com.google.protobuf.RpcChannel;
import com.google.protobuf.Service; import com.google.protobuf.Service;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat; import com.google.protobuf.TextFormat;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Protobufs utility. * Protobufs utility.
@ -820,17 +820,17 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey()); attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue())); attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
scanBuilder.addAttribute(attributeBuilder.build()); scanBuilder.addAttribute(attributeBuilder.build());
} }
} }
byte[] startRow = scan.getStartRow(); byte[] startRow = scan.getStartRow();
if (startRow != null && startRow.length > 0) { if (startRow != null && startRow.length > 0) {
scanBuilder.setStartRow(ZeroCopyLiteralByteString.wrap(startRow)); scanBuilder.setStartRow(HBaseZeroCopyByteString.wrap(startRow));
} }
byte[] stopRow = scan.getStopRow(); byte[] stopRow = scan.getStopRow();
if (stopRow != null && stopRow.length > 0) { if (stopRow != null && stopRow.length > 0) {
scanBuilder.setStopRow(ZeroCopyLiteralByteString.wrap(stopRow)); scanBuilder.setStopRow(HBaseZeroCopyByteString.wrap(stopRow));
} }
if (scan.hasFilter()) { if (scan.hasFilter()) {
scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter())); scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
@ -839,12 +839,12 @@ public final class ProtobufUtil {
Column.Builder columnBuilder = Column.newBuilder(); Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>> for (Map.Entry<byte[],NavigableSet<byte []>>
family: scan.getFamilyMap().entrySet()) { family: scan.getFamilyMap().entrySet()) {
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey())); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
NavigableSet<byte []> qualifiers = family.getValue(); NavigableSet<byte []> qualifiers = family.getValue();
columnBuilder.clearQualifier(); columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) { if (qualifiers != null && qualifiers.size() > 0) {
for (byte [] qualifier: qualifiers) { for (byte [] qualifier: qualifiers) {
columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); columnBuilder.addQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
} }
scanBuilder.addColumn(columnBuilder.build()); scanBuilder.addColumn(columnBuilder.build());
@ -952,7 +952,7 @@ public final class ProtobufUtil {
final Get get) throws IOException { final Get get) throws IOException {
ClientProtos.Get.Builder builder = ClientProtos.Get.Builder builder =
ClientProtos.Get.newBuilder(); ClientProtos.Get.newBuilder();
builder.setRow(ZeroCopyLiteralByteString.wrap(get.getRow())); builder.setRow(HBaseZeroCopyByteString.wrap(get.getRow()));
builder.setCacheBlocks(get.getCacheBlocks()); builder.setCacheBlocks(get.getCacheBlocks());
builder.setMaxVersions(get.getMaxVersions()); builder.setMaxVersions(get.getMaxVersions());
if (get.getFilter() != null) { if (get.getFilter() != null) {
@ -971,7 +971,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey()); attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue())); attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build()); builder.addAttribute(attributeBuilder.build());
} }
} }
@ -980,11 +980,11 @@ public final class ProtobufUtil {
Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap(); Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) { for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
NavigableSet<byte[]> qualifiers = family.getValue(); NavigableSet<byte[]> qualifiers = family.getValue();
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey())); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.clearQualifier(); columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) { if (qualifiers != null && qualifiers.size() > 0) {
for (byte[] qualifier: qualifiers) { for (byte[] qualifier: qualifiers) {
columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); columnBuilder.addQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
} }
builder.addColumn(columnBuilder.build()); builder.addColumn(columnBuilder.build());
@ -1013,7 +1013,7 @@ public final class ProtobufUtil {
*/ */
public static MutationProto toMutation( public static MutationProto toMutation(
final Increment increment, final MutationProto.Builder builder, long nonce) { final Increment increment, final MutationProto.Builder builder, long nonce) {
builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow())); builder.setRow(HBaseZeroCopyByteString.wrap(increment.getRow()));
builder.setMutateType(MutationType.INCREMENT); builder.setMutateType(MutationType.INCREMENT);
builder.setDurability(toDurability(increment.getDurability())); builder.setDurability(toDurability(increment.getDurability()));
if (nonce != HConstants.NO_NONCE) { if (nonce != HConstants.NO_NONCE) {
@ -1030,18 +1030,18 @@ public final class ProtobufUtil {
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder(); ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) { for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) {
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey())); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.clearQualifierValue(); columnBuilder.clearQualifierValue();
List<Cell> values = family.getValue(); List<Cell> values = family.getValue();
if (values != null && values.size() > 0) { if (values != null && values.size() > 0) {
for (Cell cell: values) { for (Cell cell: values) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap( valueBuilder.setQualifier(HBaseZeroCopyByteString.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap( valueBuilder.setValue(HBaseZeroCopyByteString.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
if (kv.getTagsLength() > 0) { if (kv.getTagsLength() > 0) {
valueBuilder.setTags(ZeroCopyLiteralByteString.wrap(kv.getTagsArray(), valueBuilder.setTags(HBaseZeroCopyByteString.wrap(kv.getTagsArray(),
kv.getTagsOffset(), kv.getTagsLength())); kv.getTagsOffset(), kv.getTagsLength()));
} }
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
@ -1054,7 +1054,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) { for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey()); attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue())); attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build()); builder.addAttribute(attributeBuilder.build());
} }
} }
@ -1095,12 +1095,12 @@ public final class ProtobufUtil {
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) { for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
columnBuilder.clear(); columnBuilder.clear();
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey())); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
for (Cell cell: family.getValue()) { for (Cell cell: family.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell); KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap( valueBuilder.setQualifier(HBaseZeroCopyByteString.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())); kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap( valueBuilder.setValue(HBaseZeroCopyByteString.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
valueBuilder.setTimestamp(kv.getTimestamp()); valueBuilder.setTimestamp(kv.getTimestamp());
if(cell.getTagsLength() > 0) { if(cell.getTagsLength() > 0) {
@ -1164,7 +1164,7 @@ public final class ProtobufUtil {
*/ */
private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type, private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
final Mutation mutation, MutationProto.Builder builder) { final Mutation mutation, MutationProto.Builder builder) {
builder.setRow(ZeroCopyLiteralByteString.wrap(mutation.getRow())); builder.setRow(HBaseZeroCopyByteString.wrap(mutation.getRow()));
builder.setMutateType(type); builder.setMutateType(type);
builder.setDurability(toDurability(mutation.getDurability())); builder.setDurability(toDurability(mutation.getDurability()));
builder.setTimestamp(mutation.getTimeStamp()); builder.setTimestamp(mutation.getTimeStamp());
@ -1173,7 +1173,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey()); attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue())); attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build()); builder.addAttribute(attributeBuilder.build());
} }
} }
@ -1304,7 +1304,7 @@ public final class ProtobufUtil {
public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) { public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) {
ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder(); ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
builder.setName(comparator.getClass().getName()); builder.setName(comparator.getClass().getName());
builder.setSerializedComparator(ZeroCopyLiteralByteString.wrap(comparator.toByteArray())); builder.setSerializedComparator(HBaseZeroCopyByteString.wrap(comparator.toByteArray()));
return builder.build(); return builder.build();
} }
@ -1366,7 +1366,7 @@ public final class ProtobufUtil {
public static FilterProtos.Filter toFilter(Filter filter) throws IOException { public static FilterProtos.Filter toFilter(Filter filter) throws IOException {
FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder(); FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder();
builder.setName(filter.getClass().getName()); builder.setName(filter.getClass().getName());
builder.setSerializedFilter(ZeroCopyLiteralByteString.wrap(filter.toByteArray())); builder.setSerializedFilter(HBaseZeroCopyByteString.wrap(filter.toByteArray()));
return builder.build(); return builder.build();
} }
@ -1855,10 +1855,10 @@ public final class ProtobufUtil {
AccessControlProtos.TablePermission.newBuilder(); AccessControlProtos.TablePermission.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName())); builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
if (tablePerm.hasFamily()) { if (tablePerm.hasFamily()) {
builder.setFamily(ZeroCopyLiteralByteString.wrap(tablePerm.getFamily())); builder.setFamily(HBaseZeroCopyByteString.wrap(tablePerm.getFamily()));
} }
if (tablePerm.hasQualifier()) { if (tablePerm.hasQualifier()) {
builder.setQualifier(ZeroCopyLiteralByteString.wrap(tablePerm.getQualifier())); builder.setQualifier(HBaseZeroCopyByteString.wrap(tablePerm.getQualifier()));
} }
Permission.Action actions[] = perm.getActions(); Permission.Action actions[] = perm.getActions();
if (actions != null) { if (actions != null) {
@ -1954,7 +1954,7 @@ public final class ProtobufUtil {
*/ */
public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) { public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
return AccessControlProtos.UserPermission.newBuilder() return AccessControlProtos.UserPermission.newBuilder()
.setUser(ZeroCopyLiteralByteString.wrap(perm.getUser())) .setUser(HBaseZeroCopyByteString.wrap(perm.getUser()))
.setPermission(toPermission(perm)) .setPermission(toPermission(perm))
.build(); .build();
} }
@ -2226,8 +2226,8 @@ public final class ProtobufUtil {
*/ */
public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) { public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder(); AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
builder.setIdentifier(ZeroCopyLiteralByteString.wrap(token.getIdentifier())); builder.setIdentifier(HBaseZeroCopyByteString.wrap(token.getIdentifier()));
builder.setPassword(ZeroCopyLiteralByteString.wrap(token.getPassword())); builder.setPassword(HBaseZeroCopyByteString.wrap(token.getPassword()));
if (token.getService() != null) { if (token.getService() != null) {
builder.setService(ByteString.copyFromUtf8(token.getService().toString())); builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
} }
@ -2324,16 +2324,16 @@ public final class ProtobufUtil {
// Doing this is going to kill us if we do it for all data passed. // Doing this is going to kill us if we do it for all data passed.
// St.Ack 20121205 // St.Ack 20121205
CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
kvbuilder.setRow(ZeroCopyLiteralByteString.wrap(kv.getRowArray(), kv.getRowOffset(), kvbuilder.setRow(HBaseZeroCopyByteString.wrap(kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength())); kv.getRowLength()));
kvbuilder.setFamily(ZeroCopyLiteralByteString.wrap(kv.getFamilyArray(), kvbuilder.setFamily(HBaseZeroCopyByteString.wrap(kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength())); kv.getFamilyOffset(), kv.getFamilyLength()));
kvbuilder.setQualifier(ZeroCopyLiteralByteString.wrap(kv.getQualifierArray(), kvbuilder.setQualifier(HBaseZeroCopyByteString.wrap(kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength())); kv.getQualifierOffset(), kv.getQualifierLength()));
kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
kvbuilder.setTimestamp(kv.getTimestamp()); kvbuilder.setTimestamp(kv.getTimestamp());
kvbuilder.setValue(ZeroCopyLiteralByteString.wrap(kv.getValueArray(), kv.getValueOffset(), kvbuilder.setValue(HBaseZeroCopyByteString.wrap(kv.getValueArray(), kv.getValueOffset(),
kv.getValueLength())); kv.getValueLength()));
return kvbuilder.build(); return kvbuilder.build();
} }
@ -2411,9 +2411,9 @@ public final class ProtobufUtil {
// input / output paths are relative to the store dir // input / output paths are relative to the store dir
// store dir is relative to region dir // store dir is relative to region dir
CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder() CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
.setTableName(ZeroCopyLiteralByteString.wrap(info.getTableName())) .setTableName(HBaseZeroCopyByteString.wrap(info.getTableName()))
.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(info.getEncodedNameAsBytes())) .setEncodedRegionName(HBaseZeroCopyByteString.wrap(info.getEncodedNameAsBytes()))
.setFamilyName(ZeroCopyLiteralByteString.wrap(family)) .setFamilyName(HBaseZeroCopyByteString.wrap(family))
.setStoreHomeDir(storeDir.getName()); //make relative .setStoreHomeDir(storeDir.getName()); //make relative
for (Path inputPath : inputPaths) { for (Path inputPath : inputPaths) {
builder.addCompactionInput(inputPath.getName()); //relative path builder.addCompactionInput(inputPath.getName()); //relative path
@ -2490,8 +2490,8 @@ public final class ProtobufUtil {
public static HBaseProtos.TableName toProtoTableName(TableName tableName) { public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
return HBaseProtos.TableName.newBuilder() return HBaseProtos.TableName.newBuilder()
.setNamespace(ZeroCopyLiteralByteString.wrap(tableName.getNamespace())) .setNamespace(HBaseZeroCopyByteString.wrap(tableName.getNamespace()))
.setQualifier(ZeroCopyLiteralByteString.wrap(tableName.getQualifier())).build(); .setQualifier(HBaseZeroCopyByteString.wrap(tableName.getQualifier())).build();
} }
public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) { public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.protobuf;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -102,7 +103,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.hbase.util.Triple;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Helper utility to build protocol buffer requests, * Helper utility to build protocol buffer requests,
@ -134,10 +134,10 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
Column.Builder columnBuilder = Column.newBuilder(); Column.Builder columnBuilder = Column.newBuilder();
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
ClientProtos.Get.Builder getBuilder = ClientProtos.Get.Builder getBuilder =
ClientProtos.Get.newBuilder(); ClientProtos.Get.newBuilder();
getBuilder.setRow(ZeroCopyLiteralByteString.wrap(row)); getBuilder.setRow(HBaseZeroCopyByteString.wrap(row));
getBuilder.addColumn(columnBuilder.build()); getBuilder.addColumn(columnBuilder.build());
getBuilder.setClosestRowBefore(true); getBuilder.setClosestRowBefore(true);
builder.setGet(getBuilder.build()); builder.setGet(getBuilder.build());
@ -182,14 +182,14 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
MutationProto.Builder mutateBuilder = MutationProto.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ZeroCopyLiteralByteString.wrap(row)); mutateBuilder.setRow(HBaseZeroCopyByteString.wrap(row));
mutateBuilder.setMutateType(MutationType.INCREMENT); mutateBuilder.setMutateType(MutationType.INCREMENT);
mutateBuilder.setDurability(ProtobufUtil.toDurability(durability)); mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder(); ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(amount))); valueBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(amount)));
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); valueBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build()); mutateBuilder.addColumnValue(columnBuilder.build());
if (nonce != HConstants.NO_NONCE) { if (nonce != HConstants.NO_NONCE) {
@ -492,7 +492,7 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder(); FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder();
for (Pair<byte[], String> familyPath: familyPaths) { for (Pair<byte[], String> familyPath: familyPaths) {
familyPathBuilder.setFamily(ZeroCopyLiteralByteString.wrap(familyPath.getFirst())); familyPathBuilder.setFamily(HBaseZeroCopyByteString.wrap(familyPath.getFirst()));
familyPathBuilder.setPath(familyPath.getSecond()); familyPathBuilder.setPath(familyPath.getSecond());
builder.addFamilyPath(familyPathBuilder.build()); builder.addFamilyPath(familyPathBuilder.build());
} }
@ -660,7 +660,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
builder.addFamily(ZeroCopyLiteralByteString.wrap(family)); builder.addFamily(HBaseZeroCopyByteString.wrap(family));
return builder.build(); return builder.build();
} }
@ -806,7 +806,7 @@ public final class RequestConverter {
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
if (splitPoint != null) { if (splitPoint != null) {
builder.setSplitPoint(ZeroCopyLiteralByteString.wrap(splitPoint)); builder.setSplitPoint(HBaseZeroCopyByteString.wrap(splitPoint));
} }
return builder.build(); return builder.build();
} }
@ -846,7 +846,7 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
builder.setMajor(major); builder.setMajor(major);
if (family != null) { if (family != null) {
builder.setFamily(ZeroCopyLiteralByteString.wrap(family)); builder.setFamily(HBaseZeroCopyByteString.wrap(family));
} }
return builder.build(); return builder.build();
} }
@ -905,7 +905,7 @@ public final class RequestConverter {
public static RegionSpecifier buildRegionSpecifier( public static RegionSpecifier buildRegionSpecifier(
final RegionSpecifierType type, final byte[] value) { final RegionSpecifierType type, final byte[] value) {
RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder(); RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder();
regionBuilder.setValue(ZeroCopyLiteralByteString.wrap(value)); regionBuilder.setValue(HBaseZeroCopyByteString.wrap(value));
regionBuilder.setType(type); regionBuilder.setType(type);
return regionBuilder.build(); return regionBuilder.build();
} }
@ -926,9 +926,9 @@ public final class RequestConverter {
final ByteArrayComparable comparator, final ByteArrayComparable comparator,
final CompareType compareType) throws IOException { final CompareType compareType) throws IOException {
Condition.Builder builder = Condition.newBuilder(); Condition.Builder builder = Condition.newBuilder();
builder.setRow(ZeroCopyLiteralByteString.wrap(row)); builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.setFamily(ZeroCopyLiteralByteString.wrap(family)); builder.setFamily(HBaseZeroCopyByteString.wrap(family));
builder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
builder.setComparator(ProtobufUtil.toComparator(comparator)); builder.setComparator(ProtobufUtil.toComparator(comparator));
builder.setCompareType(compareType); builder.setCompareType(compareType);
return builder.build(); return builder.build();
@ -960,7 +960,7 @@ public final class RequestConverter {
final TableName tableName, final byte [] columnName) { final TableName tableName, final byte [] columnName) {
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder(); DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName))); builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnName(ZeroCopyLiteralByteString.wrap(columnName)); builder.setColumnName(HBaseZeroCopyByteString.wrap(columnName));
return builder.build(); return builder.build();
} }
@ -1100,7 +1100,7 @@ public final class RequestConverter {
builder.setTableSchema(hTableDesc.convert()); builder.setTableSchema(hTableDesc.convert());
if (splitKeys != null) { if (splitKeys != null) {
for (byte [] splitKey : splitKeys) { for (byte [] splitKey : splitKeys) {
builder.addSplitKeys(ZeroCopyLiteralByteString.wrap(splitKey)); builder.addSplitKeys(HBaseZeroCopyByteString.wrap(splitKey));
} }
} }
return builder.build(); return builder.build();
@ -1253,7 +1253,7 @@ public final class RequestConverter {
public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest( public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest(
byte[] regionName) { byte[] regionName) {
return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName( return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName(
ZeroCopyLiteralByteString.wrap(regionName)).build(); HBaseZeroCopyByteString.wrap(regionName)).build();
} }
/** /**
@ -1308,10 +1308,10 @@ public final class RequestConverter {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName)); permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (family != null) { if (family != null) {
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
} }
if (qualifier != null) { if (qualifier != null) {
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
ret.setType(AccessControlProtos.Permission.Type.Table) ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder); .setTablePermission(permissionBuilder);
@ -1404,10 +1404,10 @@ public final class RequestConverter {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName)); permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
} }
if (family != null) { if (family != null) {
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
} }
if (qualifier != null) { if (qualifier != null) {
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
ret.setType(AccessControlProtos.Permission.Type.Table) ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder); .setTablePermission(permissionBuilder);

View File

@ -26,6 +26,7 @@ import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec; import javax.crypto.spec.SecretKeySpec;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -35,8 +36,6 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.protobuf.generated.EncryptionProtos; import org.apache.hadoop.hbase.protobuf.generated.EncryptionProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Some static utility methods for encryption uses in hbase-client. * Some static utility methods for encryption uses in hbase-client.
*/ */
@ -84,15 +83,15 @@ public class EncryptionUtil {
if (cipher.getIvLength() > 0) { if (cipher.getIvLength() > 0) {
iv = new byte[cipher.getIvLength()]; iv = new byte[cipher.getIvLength()];
RNG.nextBytes(iv); RNG.nextBytes(iv);
builder.setIv(ZeroCopyLiteralByteString.wrap(iv)); builder.setIv(HBaseZeroCopyByteString.wrap(iv));
} }
byte[] keyBytes = key.getEncoded(); byte[] keyBytes = key.getEncoded();
builder.setLength(keyBytes.length); builder.setLength(keyBytes.length);
builder.setHash(ZeroCopyLiteralByteString.wrap(Encryption.hash128(keyBytes))); builder.setHash(HBaseZeroCopyByteString.wrap(Encryption.hash128(keyBytes)));
ByteArrayOutputStream out = new ByteArrayOutputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream();
Encryption.encryptWithSubjectKey(out, new ByteArrayInputStream(keyBytes), subject, Encryption.encryptWithSubjectKey(out, new ByteArrayInputStream(keyBytes), subject,
conf, cipher, iv); conf, cipher, iv);
builder.setData(ZeroCopyLiteralByteString.wrap(out.toByteArray())); builder.setData(HBaseZeroCopyByteString.wrap(out.toByteArray()));
// Build and return the protobuf message // Build and return the protobuf message
out.reset(); out.reset();
builder.build().writeDelimitedTo(out); builder.build().writeDelimitedTo(out);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.security.access;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -39,7 +40,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequ
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Utility client for doing access control admin operations. * Utility client for doing access control admin operations.
@ -86,10 +86,10 @@ public class AccessControlClient {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName)); permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (family != null) { if (family != null) {
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
} }
if (qual != null) { if (qual != null) {
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qual)); permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qual));
} }
ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission( ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission(
permissionBuilder); permissionBuilder);
@ -150,10 +150,10 @@ public class AccessControlClient {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName)); permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
} }
if (family != null) { if (family != null) {
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family)); permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
} }
if (qualifier != null) { if (qualifier != null) {
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission( ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission(
permissionBuilder); permissionBuilder);

View File

@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LA
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -40,7 +41,6 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Utility client for doing visibility labels admin operations. * Utility client for doing visibility labels admin operations.
@ -86,7 +86,7 @@ public class VisibilityClient {
for (String label : labels) { for (String label : labels) {
if (label.length() > 0) { if (label.length() > 0) {
VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder(); VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
newBuilder.setLabel(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(label))); newBuilder.setLabel(HBaseZeroCopyByteString.wrap(Bytes.toBytes(label)));
builder.addVisLabel(newBuilder.build()); builder.addVisLabel(newBuilder.build());
} }
} }
@ -137,7 +137,7 @@ public class VisibilityClient {
public GetAuthsResponse call(VisibilityLabelsService service) throws IOException { public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder(); GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
getAuthReqBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(user))); getAuthReqBuilder.setUser(HBaseZeroCopyByteString.wrap(Bytes.toBytes(user)));
service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback); service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
return rpcCallback.get(); return rpcCallback.get();
} }
@ -179,10 +179,10 @@ public class VisibilityClient {
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException { public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder(); SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
setAuthReqBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(user))); setAuthReqBuilder.setUser(HBaseZeroCopyByteString.wrap(Bytes.toBytes(user)));
for (String auth : auths) { for (String auth : auths) {
if (auth.length() > 0) { if (auth.length() > 0) {
setAuthReqBuilder.addAuth(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(auth))); setAuthReqBuilder.addAuth(HBaseZeroCopyByteString.wrap(Bytes.toBytes(auth)));
} }
} }
if (setOrClear) { if (setOrClear) {

View File

@ -35,6 +35,7 @@ import java.util.Properties;
import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -70,7 +71,6 @@ import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.zookeeper.server.ZooKeeperSaslServer; import org.apache.zookeeper.server.ZooKeeperSaslServer;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Internal HBase utility class for ZooKeeper. * Internal HBase utility class for ZooKeeper.
@ -1947,7 +1947,7 @@ public class ZKUtil {
for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){ for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
byte[] columnFamilyName = e.getKey(); byte[] columnFamilyName = e.getKey();
Long curSeqId = e.getValue(); Long curSeqId = e.getValue();
storeSequenceIdBuilder.setFamilyName(ZeroCopyLiteralByteString.wrap(columnFamilyName)); storeSequenceIdBuilder.setFamilyName(HBaseZeroCopyByteString.wrap(columnFamilyName));
storeSequenceIdBuilder.setSequenceId(curSeqId); storeSequenceIdBuilder.setSequenceId(curSeqId);
regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build()); regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
storeSequenceIdBuilder.clear(); storeSequenceIdBuilder.clear();

View File

@ -33,6 +33,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.lang.NotImplementedException; import org.apache.commons.lang.NotImplementedException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -84,7 +85,6 @@ import com.google.common.base.Stopwatch;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Test client behavior w/o setting up a cluster. * Test client behavior w/o setting up a cluster.
@ -510,7 +510,7 @@ public class TestClientNoCluster extends Configured implements Tool {
if (max <= 0) break; if (max <= 0) break;
if (++count > max) break; if (++count > max) break;
HRegionInfo hri = e.getValue().getFirst(); HRegionInfo hri = e.getValue().getFirst();
ByteString row = ZeroCopyLiteralByteString.wrap(hri.getRegionName()); ByteString row = HBaseZeroCopyByteString.wrap(hri.getRegionName());
resultBuilder.clear(); resultBuilder.clear();
resultBuilder.addCell(getRegionInfo(row, hri)); resultBuilder.addCell(getRegionInfo(row, hri));
resultBuilder.addCell(getServer(row, e.getValue().getSecond())); resultBuilder.addCell(getServer(row, e.getValue().getSecond()));
@ -565,11 +565,11 @@ public class TestClientNoCluster extends Configured implements Tool {
} }
private final static ByteString CATALOG_FAMILY_BYTESTRING = private final static ByteString CATALOG_FAMILY_BYTESTRING =
ZeroCopyLiteralByteString.wrap(HConstants.CATALOG_FAMILY); HBaseZeroCopyByteString.wrap(HConstants.CATALOG_FAMILY);
private final static ByteString REGIONINFO_QUALIFIER_BYTESTRING = private final static ByteString REGIONINFO_QUALIFIER_BYTESTRING =
ZeroCopyLiteralByteString.wrap(HConstants.REGIONINFO_QUALIFIER); HBaseZeroCopyByteString.wrap(HConstants.REGIONINFO_QUALIFIER);
private final static ByteString SERVER_QUALIFIER_BYTESTRING = private final static ByteString SERVER_QUALIFIER_BYTESTRING =
ZeroCopyLiteralByteString.wrap(HConstants.SERVER_QUALIFIER); HBaseZeroCopyByteString.wrap(HConstants.SERVER_QUALIFIER);
static CellProtos.Cell.Builder getBaseCellBuilder(final ByteString row) { static CellProtos.Cell.Builder getBaseCellBuilder(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = CellProtos.Cell.newBuilder(); CellProtos.Cell.Builder cellBuilder = CellProtos.Cell.newBuilder();
@ -582,7 +582,7 @@ public class TestClientNoCluster extends Configured implements Tool {
static CellProtos.Cell getRegionInfo(final ByteString row, final HRegionInfo hri) { static CellProtos.Cell getRegionInfo(final ByteString row, final HRegionInfo hri) {
CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row); CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row);
cellBuilder.setQualifier(REGIONINFO_QUALIFIER_BYTESTRING); cellBuilder.setQualifier(REGIONINFO_QUALIFIER_BYTESTRING);
cellBuilder.setValue(ZeroCopyLiteralByteString.wrap(hri.toByteArray())); cellBuilder.setValue(HBaseZeroCopyByteString.wrap(hri.toByteArray()));
return cellBuilder.build(); return cellBuilder.build();
} }
@ -595,9 +595,9 @@ public class TestClientNoCluster extends Configured implements Tool {
static CellProtos.Cell getStartCode(final ByteString row) { static CellProtos.Cell getStartCode(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row); CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row);
cellBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(HConstants.STARTCODE_QUALIFIER)); cellBuilder.setQualifier(HBaseZeroCopyByteString.wrap(HConstants.STARTCODE_QUALIFIER));
// TODO: // TODO:
cellBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(META_SERVERNAME.getStartcode()))); cellBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(META_SERVERNAME.getStartcode())));
return cellBuilder.build(); return cellBuilder.build();
} }

View File

@ -0,0 +1,68 @@
/**
* 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 HBaseZeroCopyByteString extends LiteralByteString {
// Gotten from AsyncHBase code base with permission.
/** Private constructor so this class cannot be instantiated. */
private HBaseZeroCopyByteString() {
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);
}
/**
* Wraps a subset of a byte array in a {@link ByteString} without copying it.
*/
public static ByteString wrap(final byte[] array, int offset, int length) {
return new BoundedByteString(array, offset, length);
}
// 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 ByteString buf) {
if (buf instanceof LiteralByteString) {
return ((LiteralByteString) buf).bytes;
}
throw new UnsupportedOperationException("Need a LiteralByteString, got a "
+ buf.getClass().getName());
}
}

View File

@ -21,14 +21,12 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp. * Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp.
* Use a different codec if you want that in the stream. * Use a different codec if you want that in the stream.
@ -46,15 +44,15 @@ public class MessageCodec implements Codec {
CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder(); CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
// This copies bytes from Cell to ByteString. I don't see anyway around the copy. // This copies bytes from Cell to ByteString. I don't see anyway around the copy.
// ByteString is final. // ByteString is final.
builder.setRow(ZeroCopyLiteralByteString.wrap(cell.getRowArray(), cell.getRowOffset(), builder.setRow(HBaseZeroCopyByteString.wrap(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength())); cell.getRowLength()));
builder.setFamily(ZeroCopyLiteralByteString.wrap(cell.getFamilyArray(), cell.getFamilyOffset(), builder.setFamily(HBaseZeroCopyByteString.wrap(cell.getFamilyArray(), cell.getFamilyOffset(),
cell.getFamilyLength())); cell.getFamilyLength()));
builder.setQualifier(ZeroCopyLiteralByteString.wrap(cell.getQualifierArray(), builder.setQualifier(HBaseZeroCopyByteString.wrap(cell.getQualifierArray(),
cell.getQualifierOffset(), cell.getQualifierLength())); cell.getQualifierOffset(), cell.getQualifierLength()));
builder.setTimestamp(cell.getTimestamp()); builder.setTimestamp(cell.getTimestamp());
builder.setCellType(CellProtos.CellType.valueOf(cell.getTypeByte())); builder.setCellType(CellProtos.CellType.valueOf(cell.getTypeByte()));
builder.setValue(ZeroCopyLiteralByteString.wrap(cell.getValueArray(), cell.getValueOffset(), builder.setValue(HBaseZeroCopyByteString.wrap(cell.getValueArray(), cell.getValueOffset(),
cell.getValueLength())); cell.getValueLength()));
CellProtos.Cell pbcell = builder.build(); CellProtos.Cell pbcell = builder.build();
pbcell.writeDelimitedTo(this.out); pbcell.writeDelimitedTo(this.out);

View File

@ -24,6 +24,7 @@ import java.io.DataInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -33,9 +34,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FSProtos; import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.util.Bytes; 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 * A reference to the top or bottom half of a store file where 'bottom' is the first half
* of the file containing the keys that sort lowest and 'top' is the second half * of the file containing the keys that sort lowest and 'top' is the second half
@ -195,7 +193,7 @@ public class Reference {
FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder(); FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
builder.setRange(isTopFileRegion(getFileRegion())? builder.setRange(isTopFileRegion(getFileRegion())?
FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM); FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
builder.setSplitkey(ZeroCopyLiteralByteString.wrap(getSplitKey())); builder.setSplitkey(HBaseZeroCopyByteString.wrap(getSplitKey()));
return builder.build(); return builder.build();
} }

View File

@ -26,6 +26,7 @@ import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
@ -34,8 +35,6 @@ import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos; import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* The {@link HFile} has a fixed trailer which contains offsets to other * The {@link HFile} has a fixed trailer which contains offsets to other
* variable parts of the file. Also includes basic metadata on this file. The * variable parts of the file. Also includes basic metadata on this file. The
@ -206,7 +205,7 @@ public class FixedFileTrailer {
.setComparatorClassName(comparatorClassName) .setComparatorClassName(comparatorClassName)
.setCompressionCodec(compressionCodec.ordinal()); .setCompressionCodec(compressionCodec.ordinal());
if (encryptionKey != null) { if (encryptionKey != null) {
builder.setEncryptionKey(ZeroCopyLiteralByteString.wrap(encryptionKey)); builder.setEncryptionKey(HBaseZeroCopyByteString.wrap(encryptionKey));
} }
// We need this extra copy unfortunately to determine the final size of the // We need this extra copy unfortunately to determine the final size of the
// delimited output, see use of baos.size() below. // delimited output, see use of baos.size() below.

View File

@ -40,6 +40,7 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -69,7 +70,6 @@ import org.apache.hadoop.io.Writable;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* File format for hbase. * File format for hbase.
@ -739,8 +739,8 @@ public class HFile {
HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder(); HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) { for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder(); HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey())); bbpBuilder.setFirst(HBaseZeroCopyByteString.wrap(e.getKey()));
bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue())); bbpBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue()));
builder.addMapEntry(bbpBuilder.build()); builder.addMapEntry(bbpBuilder.build());
} }
out.write(ProtobufUtil.PB_MAGIC); out.write(ProtobufUtil.PB_MAGIC);

View File

@ -28,6 +28,7 @@ import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.UUID; import java.util.UUID;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -69,7 +70,6 @@ import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptContext;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job * TableSnapshotInputFormat allows a MapReduce job to run over a table snapshot. The job
@ -160,7 +160,7 @@ public class TableSnapshotInputFormat extends InputFormat<ImmutableBytesWritable
MapReduceProtos.TableSnapshotRegionSplit.newBuilder() MapReduceProtos.TableSnapshotRegionSplit.newBuilder()
.setRegion(RegionSpecifier.newBuilder() .setRegion(RegionSpecifier.newBuilder()
.setType(RegionSpecifierType.ENCODED_REGION_NAME) .setType(RegionSpecifierType.ENCODED_REGION_NAME)
.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(regionName))).build()); .setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(regionName))).build());
for (String location : locations) { for (String location : locations) {
builder.addLocations(location); builder.addLocations(location);

View File

@ -28,6 +28,7 @@ import java.util.Map;
import java.util.NavigableMap; import java.util.NavigableMap;
import java.util.UUID; import java.util.UUID;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.ZeroCopyLiteralByteString;
@InterfaceAudience.Private @InterfaceAudience.Private
public class ReplicationProtbufUtil { public class ReplicationProtbufUtil {
@ -91,8 +91,8 @@ public class ReplicationProtbufUtil {
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder(); WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
HLogKey key = entry.getKey(); HLogKey key = entry.getKey();
keyBuilder.setEncodedRegionName( keyBuilder.setEncodedRegionName(
ZeroCopyLiteralByteString.wrap(key.getEncodedRegionName())); HBaseZeroCopyByteString.wrap(key.getEncodedRegionName()));
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName())); keyBuilder.setTableName(HBaseZeroCopyByteString.wrap(key.getTablename().getName()));
keyBuilder.setLogSequenceNumber(key.getLogSeqNum()); keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
keyBuilder.setWriteTime(key.getWriteTime()); keyBuilder.setWriteTime(key.getWriteTime());
if (key.getNonce() != HConstants.NO_NONCE) { if (key.getNonce() != HConstants.NO_NONCE) {
@ -110,7 +110,7 @@ public class ReplicationProtbufUtil {
NavigableMap<byte[], Integer> scopes = key.getScopes(); NavigableMap<byte[], Integer> scopes = key.getScopes();
if (scopes != null && !scopes.isEmpty()) { if (scopes != null && !scopes.isEmpty()) {
for (Map.Entry<byte[], Integer> scope: scopes.entrySet()) { for (Map.Entry<byte[], Integer> scope: scopes.entrySet()) {
scopeBuilder.setFamily(ZeroCopyLiteralByteString.wrap(scope.getKey())); scopeBuilder.setFamily(HBaseZeroCopyByteString.wrap(scope.getKey()));
WALProtos.ScopeType scopeType = WALProtos.ScopeType scopeType =
WALProtos.ScopeType.valueOf(scope.getValue().intValue()); WALProtos.ScopeType.valueOf(scope.getValue().intValue());
scopeBuilder.setScopeType(scopeType); scopeBuilder.setScopeType(scopeType);

View File

@ -50,6 +50,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.management.ObjectName; import javax.management.ObjectName;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -194,7 +195,6 @@ import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@ -239,7 +239,6 @@ import com.google.protobuf.Message;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat; import com.google.protobuf.TextFormat;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* HRegionServer makes a set of HRegions available to clients. It checks in with * HRegionServer makes a set of HRegions available to clients. It checks in with
@ -1340,7 +1339,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
RegionLoad.Builder regionLoad = RegionLoad.newBuilder(); RegionLoad.Builder regionLoad = RegionLoad.newBuilder();
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
regionSpecifier.setType(RegionSpecifierType.REGION_NAME); regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
regionSpecifier.setValue(ZeroCopyLiteralByteString.wrap(name)); regionSpecifier.setValue(HBaseZeroCopyByteString.wrap(name));
regionLoad.setRegionSpecifier(regionSpecifier.build()) regionLoad.setRegionSpecifier(regionSpecifier.build())
.setStores(stores) .setStores(stores)
.setStorefiles(storefiles) .setStorefiles(storefiles)
@ -4005,7 +4004,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder(); RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
if (regionsToFlush != null) { if (regionsToFlush != null) {
for (byte[] region: regionsToFlush) { for (byte[] region: regionsToFlush) {
builder.addRegionToFlush(ZeroCopyLiteralByteString.wrap(region)); builder.addRegionToFlush(HBaseZeroCopyByteString.wrap(region));
} }
} }
return builder.build(); return builder.build();

View File

@ -31,6 +31,7 @@ import java.util.NavigableMap;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.UUID; import java.util.UUID;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -47,7 +48,6 @@ import org.apache.hadoop.io.WritableUtils;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* A Key for an entry in the change log. * A Key for an entry in the change log.
@ -451,8 +451,8 @@ public class HLogKey implements WritableComparable<HLogKey> {
throws IOException { throws IOException {
WALKey.Builder builder = WALKey.newBuilder(); WALKey.Builder builder = WALKey.newBuilder();
if (compressionContext == null) { if (compressionContext == null) {
builder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(this.encodedRegionName)); builder.setEncodedRegionName(HBaseZeroCopyByteString.wrap(this.encodedRegionName));
builder.setTableName(ZeroCopyLiteralByteString.wrap(this.tablename.getName())); builder.setTableName(HBaseZeroCopyByteString.wrap(this.tablename.getName()));
} else { } else {
builder.setEncodedRegionName(compressor.compress(this.encodedRegionName, builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
compressionContext.regionDict)); compressionContext.regionDict));
@ -476,7 +476,7 @@ public class HLogKey implements WritableComparable<HLogKey> {
if (scopes != null) { if (scopes != null) {
for (Map.Entry<byte[], Integer> e : scopes.entrySet()) { for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
ByteString family = (compressionContext == null) ? ByteString family = (compressionContext == null) ?
ZeroCopyLiteralByteString.wrap(e.getKey()) HBaseZeroCopyByteString.wrap(e.getKey())
: compressor.compress(e.getKey(), compressionContext.familyDict); : compressor.compress(e.getKey(), compressionContext.familyDict);
builder.addScopes(FamilyScope.newBuilder() builder.addScopes(FamilyScope.newBuilder()
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue()))); .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));

View File

@ -24,6 +24,7 @@ import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec; import javax.crypto.spec.SecretKeySpec;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -34,8 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import com.google.protobuf.ZeroCopyLiteralByteString;
public class SecureProtobufLogWriter extends ProtobufLogWriter { public class SecureProtobufLogWriter extends ProtobufLogWriter {
private static final Log LOG = LogFactory.getLog(SecureProtobufLogWriter.class); private static final Log LOG = LogFactory.getLog(SecureProtobufLogWriter.class);
@ -58,11 +57,11 @@ public class SecureProtobufLogWriter extends ProtobufLogWriter {
byte[] keyBytes = new byte[cipher.getKeyLength()]; byte[] keyBytes = new byte[cipher.getKeyLength()];
rng.nextBytes(keyBytes); rng.nextBytes(keyBytes);
Key key = new SecretKeySpec(keyBytes, cipher.getName()); Key key = new SecretKeySpec(keyBytes, cipher.getName());
builder.setEncryptionKey(ZeroCopyLiteralByteString.wrap(EncryptionUtil.wrapKey(conf, builder.setEncryptionKey(HBaseZeroCopyByteString.wrap(EncryptionUtil.wrapKey(conf,
conf.get(HConstants.CRYPTO_WAL_KEY_NAME_CONF_KEY, conf.get(HConstants.CRYPTO_WAL_KEY_NAME_CONF_KEY,
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
User.getCurrent().getShortName())), User.getCurrent().getShortName())),
key))); key)));
// Set up the encryptor // Set up the encryptor
encryptor = cipher.getEncryptor(); encryptor = cipher.getEncryptor();

View File

@ -28,6 +28,7 @@ import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlValue; import javax.xml.bind.annotation.XmlValue;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
@ -35,8 +36,6 @@ import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell; import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell;
import org.codehaus.jackson.annotate.JsonProperty; 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 * 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- * optional qualifier, and either the timestamp when it was stored or the user-
@ -185,8 +184,8 @@ public class CellModel implements ProtobufMessageHandler, Serializable {
@Override @Override
public byte[] createProtobufOutput() { public byte[] createProtobufOutput() {
Cell.Builder builder = Cell.newBuilder(); Cell.Builder builder = Cell.newBuilder();
builder.setColumn(ZeroCopyLiteralByteString.wrap(getColumn())); builder.setColumn(HBaseZeroCopyByteString.wrap(getColumn()));
builder.setData(ZeroCopyLiteralByteString.wrap(getValue())); builder.setData(HBaseZeroCopyByteString.wrap(getValue()));
if (hasUserTimestamp()) { if (hasUserTimestamp()) {
builder.setTimestamp(getTimestamp()); builder.setTimestamp(getTimestamp());
} }

View File

@ -29,14 +29,13 @@ import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell; import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell;
import org.apache.hadoop.hbase.rest.protobuf.generated.CellSetMessage.CellSet; import org.apache.hadoop.hbase.rest.protobuf.generated.CellSetMessage.CellSet;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Representation of a grouping of cells. May contain cells from more than * Representation of a grouping of cells. May contain cells from more than
* one row. Encapsulates RowModel and CellModel models. * one row. Encapsulates RowModel and CellModel models.
@ -115,11 +114,11 @@ public class CellSetModel implements Serializable, ProtobufMessageHandler {
CellSet.Builder builder = CellSet.newBuilder(); CellSet.Builder builder = CellSet.newBuilder();
for (RowModel row: getRows()) { for (RowModel row: getRows()) {
CellSet.Row.Builder rowBuilder = CellSet.Row.newBuilder(); CellSet.Row.Builder rowBuilder = CellSet.Row.newBuilder();
rowBuilder.setKey(ZeroCopyLiteralByteString.wrap(row.getKey())); rowBuilder.setKey(HBaseZeroCopyByteString.wrap(row.getKey()));
for (CellModel cell: row.getCells()) { for (CellModel cell: row.getCells()) {
Cell.Builder cellBuilder = Cell.newBuilder(); Cell.Builder cellBuilder = Cell.newBuilder();
cellBuilder.setColumn(ZeroCopyLiteralByteString.wrap(cell.getColumn())); cellBuilder.setColumn(HBaseZeroCopyByteString.wrap(cell.getColumn()));
cellBuilder.setData(ZeroCopyLiteralByteString.wrap(cell.getValue())); cellBuilder.setData(HBaseZeroCopyByteString.wrap(cell.getValue()));
if (cell.hasUserTimestamp()) { if (cell.hasUserTimestamp()) {
cellBuilder.setTimestamp(cell.getTimestamp()); cellBuilder.setTimestamp(cell.getTimestamp());
} }

View File

@ -32,6 +32,7 @@ import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -74,7 +75,6 @@ import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
import com.sun.jersey.api.json.JSONConfiguration; import com.sun.jersey.api.json.JSONConfiguration;
import com.sun.jersey.api.json.JSONJAXBContext; import com.sun.jersey.api.json.JSONJAXBContext;
import com.sun.jersey.api.json.JSONMarshaller; import com.sun.jersey.api.json.JSONMarshaller;
@ -759,13 +759,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
public byte[] createProtobufOutput() { public byte[] createProtobufOutput() {
Scanner.Builder builder = Scanner.newBuilder(); Scanner.Builder builder = Scanner.newBuilder();
if (!Bytes.equals(startRow, HConstants.EMPTY_START_ROW)) { if (!Bytes.equals(startRow, HConstants.EMPTY_START_ROW)) {
builder.setStartRow(ZeroCopyLiteralByteString.wrap(startRow)); builder.setStartRow(HBaseZeroCopyByteString.wrap(startRow));
} }
if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) { if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) {
builder.setEndRow(ZeroCopyLiteralByteString.wrap(endRow)); builder.setEndRow(HBaseZeroCopyByteString.wrap(endRow));
} }
for (byte[] column: columns) { for (byte[] column: columns) {
builder.addColumns(ZeroCopyLiteralByteString.wrap(column)); builder.addColumns(HBaseZeroCopyByteString.wrap(column));
} }
if (startTime != 0) { if (startTime != 0) {
builder.setStartTime(startTime); builder.setStartTime(startTime);

View File

@ -29,13 +29,12 @@ import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlElementWrapper; import javax.xml.bind.annotation.XmlElementWrapper;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus; import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Representation of the status of a storage cluster: * Representation of the status of a storage cluster:
* <p> * <p>
@ -722,7 +721,7 @@ public class StorageClusterStatusModel
for (Node.Region region: node.regions) { for (Node.Region region: node.regions) {
StorageClusterStatus.Region.Builder regionBuilder = StorageClusterStatus.Region.Builder regionBuilder =
StorageClusterStatus.Region.newBuilder(); StorageClusterStatus.Region.newBuilder();
regionBuilder.setName(ZeroCopyLiteralByteString.wrap(region.name)); regionBuilder.setName(HBaseZeroCopyByteString.wrap(region.name));
regionBuilder.setStores(region.stores); regionBuilder.setStores(region.stores);
regionBuilder.setStorefiles(region.storefiles); regionBuilder.setStorefiles(region.storefiles);
regionBuilder.setStorefileSizeMB(region.storefileSizeMB); regionBuilder.setStorefileSizeMB(region.storefileSizeMB);

View File

@ -28,12 +28,11 @@ import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.TableInfoMessage.TableInfo; import org.apache.hadoop.hbase.rest.protobuf.generated.TableInfoMessage.TableInfo;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Representation of a list of table regions. * Representation of a list of table regions.
* *
@ -135,8 +134,8 @@ public class TableInfoModel implements Serializable, ProtobufMessageHandler {
TableInfo.Region.Builder regionBuilder = TableInfo.Region.newBuilder(); TableInfo.Region.Builder regionBuilder = TableInfo.Region.newBuilder();
regionBuilder.setName(aRegion.getName()); regionBuilder.setName(aRegion.getName());
regionBuilder.setId(aRegion.getId()); regionBuilder.setId(aRegion.getId());
regionBuilder.setStartKey(ZeroCopyLiteralByteString.wrap(aRegion.getStartKey())); regionBuilder.setStartKey(HBaseZeroCopyByteString.wrap(aRegion.getStartKey()));
regionBuilder.setEndKey(ZeroCopyLiteralByteString.wrap(aRegion.getEndKey())); regionBuilder.setEndKey(HBaseZeroCopyByteString.wrap(aRegion.getEndKey()));
regionBuilder.setLocation(aRegion.getLocation()); regionBuilder.setLocation(aRegion.getLocation());
builder.addRegions(regionBuilder); builder.addRegions(regionBuilder);
} }

View File

@ -35,6 +35,7 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -116,7 +117,6 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.Service; import com.google.protobuf.Service;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in * Coprocessor that has both the MasterObserver and RegionObserver implemented that supports in
@ -1239,7 +1239,7 @@ public class VisibilityController extends BaseRegionObserver implements MasterOb
try { try {
List<String> labels = getUserAuthsFromLabelsTable(user); List<String> labels = getUserAuthsFromLabelsTable(user);
for (String label : labels) { for (String label : labels) {
response.addAuth(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(label))); response.addAuth(HBaseZeroCopyByteString.wrap(Bytes.toBytes(label)));
} }
} catch (IOException e) { } catch (IOException e) {
ResponseConverter.setControllerException(controller, e); ResponseConverter.setControllerException(controller, e);

View File

@ -21,6 +21,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagType;
@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Utility method to support visibility * Utility method to support visibility
@ -56,7 +56,7 @@ public class VisibilityUtils {
VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder(); VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
for (Entry<String, Integer> entry : existingLabels.entrySet()) { for (Entry<String, Integer> entry : existingLabels.entrySet()) {
VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder(); VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
visLabBuilder.setLabel(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(entry.getKey()))); visLabBuilder.setLabel(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
visLabBuilder.setOrdinal(entry.getValue()); visLabBuilder.setOrdinal(entry.getValue());
visReqBuilder.addVisLabel(visLabBuilder.build()); visReqBuilder.addVisLabel(visLabBuilder.build());
} }
@ -72,7 +72,7 @@ public class VisibilityUtils {
MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder(); MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
for (Entry<String, List<Integer>> entry : userAuths.entrySet()) { for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder(); UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
userAuthsBuilder.setUser(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(entry.getKey()))); userAuthsBuilder.setUser(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
for (Integer label : entry.getValue()) { for (Integer label : entry.getValue()) {
userAuthsBuilder.addAuth(label); userAuthsBuilder.addAuth(label);
} }

View File

@ -30,6 +30,7 @@ import java.util.Map;
import java.util.NavigableMap; import java.util.NavigableMap;
import java.util.TreeMap; import java.util.TreeMap;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -59,7 +60,6 @@ import org.junit.experimental.categories.Category;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* TestEndpoint: test cases to verify coprocessor Endpoint * TestEndpoint: test cases to verify coprocessor Endpoint
@ -125,9 +125,9 @@ public class TestCoprocessorEndpoint {
new BlockingRpcCallback<ColumnAggregationProtos.SumResponse>(); new BlockingRpcCallback<ColumnAggregationProtos.SumResponse>();
ColumnAggregationProtos.SumRequest.Builder builder = ColumnAggregationProtos.SumRequest.Builder builder =
ColumnAggregationProtos.SumRequest.newBuilder(); ColumnAggregationProtos.SumRequest.newBuilder();
builder.setFamily(ZeroCopyLiteralByteString.wrap(family)); builder.setFamily(HBaseZeroCopyByteString.wrap(family));
if (qualifier != null && qualifier.length > 0) { if (qualifier != null && qualifier.length > 0) {
builder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
} }
instance.sum(null, builder.build(), rpcCallback); instance.sum(null, builder.build(), rpcCallback);
return rpcCallback.get().getSum(); return rpcCallback.get().getSum();

View File

@ -33,6 +33,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
@ -70,7 +71,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import com.google.protobuf.Message; import com.google.protobuf.Message;
import com.google.protobuf.ZeroCopyLiteralByteString;
import com.sun.org.apache.commons.logging.Log; import com.sun.org.apache.commons.logging.Log;
import com.sun.org.apache.commons.logging.LogFactory; import com.sun.org.apache.commons.logging.LogFactory;
@ -362,7 +362,7 @@ public class TestRowProcessorEndpoint {
public IncCounterProcessorRequest getRequestData() throws IOException { public IncCounterProcessorRequest getRequestData() throws IOException {
IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder(); IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
builder.setCounter(counter); builder.setCounter(counter);
builder.setRow(ZeroCopyLiteralByteString.wrap(row)); builder.setRow(HBaseZeroCopyByteString.wrap(row));
return builder.build(); return builder.build();
} }
@ -441,8 +441,8 @@ public class TestRowProcessorEndpoint {
public FriendsOfFriendsProcessorRequest getRequestData() throws IOException { public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
FriendsOfFriendsProcessorRequest.Builder builder = FriendsOfFriendsProcessorRequest.Builder builder =
FriendsOfFriendsProcessorRequest.newBuilder(); FriendsOfFriendsProcessorRequest.newBuilder();
builder.setPerson(ZeroCopyLiteralByteString.wrap(person)); builder.setPerson(HBaseZeroCopyByteString.wrap(person));
builder.setRow(ZeroCopyLiteralByteString.wrap(row)); builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.addAllResult(result); builder.addAllResult(result);
FriendsOfFriendsProcessorRequest f = builder.build(); FriendsOfFriendsProcessorRequest f = builder.build();
return f; return f;
@ -546,8 +546,8 @@ public class TestRowProcessorEndpoint {
@Override @Override
public RowSwapProcessorRequest getRequestData() throws IOException { public RowSwapProcessorRequest getRequestData() throws IOException {
RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder(); RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
builder.setRow1(ZeroCopyLiteralByteString.wrap(row1)); builder.setRow1(HBaseZeroCopyByteString.wrap(row1));
builder.setRow2(ZeroCopyLiteralByteString.wrap(row2)); builder.setRow2(HBaseZeroCopyByteString.wrap(row2));
return builder.build(); return builder.build();
} }
@ -606,7 +606,7 @@ public class TestRowProcessorEndpoint {
@Override @Override
public TimeoutProcessorRequest getRequestData() throws IOException { public TimeoutProcessorRequest getRequestData() throws IOException {
TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder(); TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
builder.setRow(ZeroCopyLiteralByteString.wrap(row)); builder.setRow(HBaseZeroCopyByteString.wrap(row));
return builder.build(); return builder.build();
} }

View File

@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException; import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
@ -43,7 +44,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Class to test ProtobufUtil. * Class to test ProtobufUtil.
@ -55,12 +55,12 @@ public class TestProtobufUtil {
NameBytesPair.Builder builder = NameBytesPair.newBuilder(); NameBytesPair.Builder builder = NameBytesPair.newBuilder();
final String omg = "OMG!!!"; final String omg = "OMG!!!";
builder.setName("java.io.IOException"); builder.setName("java.io.IOException");
builder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(omg))); builder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(omg)));
Throwable t = ProtobufUtil.toException(builder.build()); Throwable t = ProtobufUtil.toException(builder.build());
assertEquals(omg, t.getMessage()); assertEquals(omg, t.getMessage());
builder.clear(); builder.clear();
builder.setName("org.apache.hadoop.ipc.RemoteException"); builder.setName("org.apache.hadoop.ipc.RemoteException");
builder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(omg))); builder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(omg)));
t = ProtobufUtil.toException(builder.build()); t = ProtobufUtil.toException(builder.build());
assertEquals(omg, t.getMessage()); assertEquals(omg, t.getMessage());
} }
@ -204,10 +204,10 @@ public class TestProtobufUtil {
valueBuilder.setFamily(ByteString.copyFromUtf8("f1")); valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder(); QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1")); qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
qualifierBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(11L))); qualifierBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(11L)));
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2")); qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
qualifierBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(22L))); qualifierBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(22L)));
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build()); mutateBuilder.addColumnValue(valueBuilder.build());

View File

@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -42,7 +43,7 @@ import org.junit.experimental.categories.Category;
import org.mockito.Mockito; import org.mockito.Mockito;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString;
/** /**
* Tests that verify certain RPCs get a higher QoS. * Tests that verify certain RPCs get a higher QoS.
*/ */
@ -70,12 +71,12 @@ public class TestPriorityRpc {
GetRequest.Builder getRequestBuilder = GetRequest.newBuilder(); GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder(); RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME); regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
ByteString name = ZeroCopyLiteralByteString.wrap(HRegionInfo.FIRST_META_REGIONINFO.getRegionName()); ByteString name = HBaseZeroCopyByteString.wrap(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
regionSpecifierBuilder.setValue(name); regionSpecifierBuilder.setValue(name);
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build(); RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
getRequestBuilder.setRegion(regionSpecifier); getRequestBuilder.setRegion(regionSpecifier);
Get.Builder getBuilder = Get.newBuilder(); Get.Builder getBuilder = Get.newBuilder();
getBuilder.setRow(ZeroCopyLiteralByteString.wrap("somerow".getBytes())); getBuilder.setRow(HBaseZeroCopyByteString.wrap("somerow".getBytes()));
getRequestBuilder.setGet(getBuilder.build()); getRequestBuilder.setGet(getBuilder.build());
GetRequest getRequest = getRequestBuilder.build(); GetRequest getRequest = getRequestBuilder.build();
RequestHeader header = headerBuilder.build(); RequestHeader header = headerBuilder.build();

View File

@ -25,6 +25,7 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -50,8 +51,6 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import com.google.protobuf.ZeroCopyLiteralByteString;
@Category(MediumTests.class) @Category(MediumTests.class)
public class TestReplicationSink { public class TestReplicationSink {
private static final Log LOG = LogFactory.getLog(TestReplicationSink.class); private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
@ -262,9 +261,9 @@ public class TestReplicationSink {
uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits()); uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits()); uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
keyBuilder.setClusterId(uuidBuilder.build()); keyBuilder.setClusterId(uuidBuilder.build());
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(table)); keyBuilder.setTableName(HBaseZeroCopyByteString.wrap(table));
keyBuilder.setWriteTime(now); keyBuilder.setWriteTime(now);
keyBuilder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(HConstants.EMPTY_BYTE_ARRAY)); keyBuilder.setEncodedRegionName(HBaseZeroCopyByteString.wrap(HConstants.EMPTY_BYTE_ARRAY));
keyBuilder.setLogSequenceNumber(-1); keyBuilder.setLogSequenceNumber(-1);
builder.setKey(keyBuilder.build()); builder.setKey(keyBuilder.build());
cells.add(kv); cells.add(kv);