HBASE-9867 Save on array copies with a subclass of LiteralByteString
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1537750 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9233b405ba
commit
dcf61670d4
|
@ -19,7 +19,13 @@
|
|||
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
|
@ -34,12 +40,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Re
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.io.VersionedWritable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -334,7 +335,7 @@ public class ClusterStatus extends VersionedWritable {
|
|||
ClusterStatusProtos.RegionState rs = rit.getValue().convert();
|
||||
RegionSpecifier.Builder spec =
|
||||
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
|
||||
spec.setValue(ByteString.copyFrom(Bytes.toBytes(rit.getKey())));
|
||||
spec.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(rit.getKey())));
|
||||
|
||||
RegionInTransition pbRIT =
|
||||
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
|
||||
|
|
|
@ -19,8 +19,9 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -1230,11 +1231,11 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
|
|||
*/
|
||||
public ColumnFamilySchema convert() {
|
||||
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
|
||||
builder.setName(ByteString.copyFrom(getName()));
|
||||
builder.setName(ZeroCopyLiteralByteString.wrap(getName()));
|
||||
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
|
||||
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
|
||||
aBuilder.setFirst(ByteString.copyFrom(e.getKey().get()));
|
||||
aBuilder.setSecond(ByteString.copyFrom(e.getValue().get()));
|
||||
aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get()));
|
||||
aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get()));
|
||||
builder.addAttributes(aBuilder.build());
|
||||
}
|
||||
for (Map.Entry<String, String> e : this.configuration.entrySet()) {
|
||||
|
|
|
@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* HRegion information.
|
||||
|
@ -843,10 +843,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
|
||||
builder.setRegionId(info.getRegionId());
|
||||
if (info.getStartKey() != null) {
|
||||
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
|
||||
builder.setStartKey(ZeroCopyLiteralByteString.wrap(info.getStartKey()));
|
||||
}
|
||||
if (info.getEndKey() != null) {
|
||||
builder.setEndKey(ByteString.copyFrom(info.getEndKey()));
|
||||
builder.setEndKey(ZeroCopyLiteralByteString.wrap(info.getEndKey()));
|
||||
}
|
||||
builder.setOffline(info.isOffline());
|
||||
builder.setSplit(info.isSplit());
|
||||
|
|
|
@ -53,8 +53,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* HTableDescriptor contains the details about an HBase table such as the descriptors of
|
||||
|
@ -1435,8 +1435,8 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
|
|||
builder.setTableName(ProtobufUtil.toProtoTableName(getTableName()));
|
||||
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
|
||||
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
|
||||
aBuilder.setFirst(ByteString.copyFrom(e.getKey().get()));
|
||||
aBuilder.setSecond(ByteString.copyFrom(e.getValue().get()));
|
||||
aBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey().get()));
|
||||
aBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue().get()));
|
||||
builder.addAttributes(aBuilder.build());
|
||||
}
|
||||
for (HColumnDescriptor hcd: getColumnFamilies()) {
|
||||
|
|
|
@ -17,8 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
|
@ -103,10 +104,10 @@ public class RegionTransition {
|
|||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
|
||||
setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
|
||||
ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
|
||||
setEventTypeCode(type.getCode()).setRegionName(ByteString.copyFrom(regionName)).
|
||||
setEventTypeCode(type.getCode()).setRegionName(ZeroCopyLiteralByteString.wrap(regionName)).
|
||||
setServerName(pbsn);
|
||||
builder.setCreateTime(System.currentTimeMillis());
|
||||
if (payload != null) builder.setPayload(ByteString.copyFrom(payload));
|
||||
if (payload != null) builder.setPayload(ZeroCopyLiteralByteString.wrap(payload));
|
||||
return new RegionTransition(builder.build());
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* ColumnInterpreter for doing Aggregation's with BigDecimal columns. This class
|
||||
|
@ -121,7 +121,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
|
|||
|
||||
private BigDecimalMsg getProtoForType(BigDecimal t) {
|
||||
BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder();
|
||||
return builder.setBigdecimalMsg(ByteString.copyFrom(Bytes.toBytes(t))).build();
|
||||
return builder.setBigdecimalMsg(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(t))).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
|
|||
import static org.apache.hadoop.hbase.HConstants.LAST_ROW;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -136,8 +137,8 @@ public class SecureBulkLoadClient {
|
|||
if(userToken != null) {
|
||||
protoDT =
|
||||
SecureBulkLoadProtos.DelegationToken.newBuilder()
|
||||
.setIdentifier(ByteString.copyFrom(userToken.getIdentifier()))
|
||||
.setPassword(ByteString.copyFrom(userToken.getPassword()))
|
||||
.setIdentifier(ZeroCopyLiteralByteString.wrap(userToken.getIdentifier()))
|
||||
.setPassword(ZeroCopyLiteralByteString.wrap(userToken.getPassword()))
|
||||
.setKind(userToken.getKind().toString())
|
||||
.setService(userToken.getService().toString()).build();
|
||||
}
|
||||
|
@ -146,7 +147,7 @@ public class SecureBulkLoadClient {
|
|||
new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>();
|
||||
for(Pair<byte[], String> el: familyPaths) {
|
||||
protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder()
|
||||
.setFamily(ByteString.copyFrom(el.getFirst()))
|
||||
.setFamily(ZeroCopyLiteralByteString.wrap(el.getFirst()))
|
||||
.setPath(el.getSecond()).build());
|
||||
}
|
||||
|
||||
|
|
|
@ -18,13 +18,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
|
||||
/** Base class for byte array comparators */
|
||||
@InterfaceAudience.Public
|
||||
|
@ -53,7 +54,7 @@ public abstract class ByteArrayComparable implements Comparable<byte[]> {
|
|||
ComparatorProtos.ByteArrayComparable convert() {
|
||||
ComparatorProtos.ByteArrayComparable.Builder builder =
|
||||
ComparatorProtos.ByteArrayComparable.newBuilder();
|
||||
if (value != null) builder.setValue(ByteString.copyFrom(value));
|
||||
if (value != null) builder.setValue(ZeroCopyLiteralByteString.wrap(value));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -18,9 +18,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -29,7 +28,9 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
|
||||
|
@ -174,7 +175,7 @@ public class ColumnPaginationFilter extends FilterBase
|
|||
builder.setOffset(this.offset);
|
||||
}
|
||||
if (this.columnOffset != null) {
|
||||
builder.setColumnOffset(ByteString.copyFrom(this.columnOffset));
|
||||
builder.setColumnOffset(ZeroCopyLiteralByteString.wrap(this.columnOffset));
|
||||
}
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
|
|
@ -19,9 +19,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -30,7 +29,9 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* This filter is used for selecting only those keys with columns that matches
|
||||
|
@ -94,7 +95,7 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
public byte [] toByteArray() {
|
||||
FilterProtos.ColumnPrefixFilter.Builder builder =
|
||||
FilterProtos.ColumnPrefixFilter.newBuilder();
|
||||
if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
|
||||
if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix));
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
||||
|
|
|
@ -22,8 +22,9 @@ package org.apache.hadoop.hbase.filter;
|
|||
import static org.apache.hadoop.hbase.util.Bytes.len;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -172,9 +173,9 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
public byte [] toByteArray() {
|
||||
FilterProtos.ColumnRangeFilter.Builder builder =
|
||||
FilterProtos.ColumnRangeFilter.newBuilder();
|
||||
if (this.minColumn != null) builder.setMinColumn(ByteString.copyFrom(this.minColumn));
|
||||
if (this.minColumn != null) builder.setMinColumn(ZeroCopyLiteralByteString.wrap(this.minColumn));
|
||||
builder.setMinColumnInclusive(this.minColumnInclusive);
|
||||
if (this.maxColumn != null) builder.setMaxColumn(ByteString.copyFrom(this.maxColumn));
|
||||
if (this.maxColumn != null) builder.setMaxColumn(ZeroCopyLiteralByteString.wrap(this.maxColumn));
|
||||
builder.setMaxColumnInclusive(this.maxColumnInclusive);
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
|
|
@ -36,8 +36,8 @@ import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* A filter for adding inter-column timestamp matching
|
||||
|
@ -225,10 +225,10 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
FilterProtos.DependentColumnFilter.newBuilder();
|
||||
builder.setCompareFilter(super.convert());
|
||||
if (this.columnFamily != null) {
|
||||
builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
|
||||
builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily));
|
||||
}
|
||||
if (this.columnQualifier != null) {
|
||||
builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
|
||||
builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier));
|
||||
}
|
||||
builder.setDropDependentColumn(this.dropDependentColumn);
|
||||
return builder.build().toByteArray();
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -87,7 +89,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
|
|||
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
|
||||
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
|
||||
for (byte[] qualifier : qualifiers) {
|
||||
if (qualifier != null) builder.addQualifiers(ByteString.copyFrom(qualifier));
|
||||
if (qualifier != null) builder.addQualifiers(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
|
|
@ -17,8 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -146,8 +147,8 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
FilterProtos.FuzzyRowFilter.newBuilder();
|
||||
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
|
||||
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
|
||||
bbpBuilder.setFirst(ByteString.copyFrom(fuzzyData.getFirst()));
|
||||
bbpBuilder.setSecond(ByteString.copyFrom(fuzzyData.getSecond()));
|
||||
bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(fuzzyData.getFirst()));
|
||||
bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(fuzzyData.getSecond()));
|
||||
builder.addFuzzyKeysData(bbpBuilder);
|
||||
}
|
||||
return builder.build().toByteArray();
|
||||
|
|
|
@ -19,16 +19,17 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* A Filter that stops after the given row. There is no "RowStopFilter" because
|
||||
|
@ -85,7 +86,7 @@ public class InclusiveStopFilter extends FilterBase {
|
|||
public byte [] toByteArray() {
|
||||
FilterProtos.InclusiveStopFilter.Builder builder =
|
||||
FilterProtos.InclusiveStopFilter.newBuilder();
|
||||
if (this.stopRowKey != null) builder.setStopRowKey(ByteString.copyFrom(this.stopRowKey));
|
||||
if (this.stopRowKey != null) builder.setStopRowKey(ZeroCopyLiteralByteString.wrap(this.stopRowKey));
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
||||
|
|
|
@ -17,8 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -112,7 +114,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
FilterProtos.MultipleColumnPrefixFilter.Builder builder =
|
||||
FilterProtos.MultipleColumnPrefixFilter.newBuilder();
|
||||
for (byte [] element : sortedPrefixes) {
|
||||
if (element != null) builder.addSortedPrefixes(ByteString.copyFrom(element));
|
||||
if (element != null) builder.addSortedPrefixes(ZeroCopyLiteralByteString.wrap(element));
|
||||
}
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
|
|
@ -20,8 +20,9 @@
|
|||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -90,7 +91,7 @@ public class PrefixFilter extends FilterBase {
|
|||
public byte [] toByteArray() {
|
||||
FilterProtos.PrefixFilter.Builder builder =
|
||||
FilterProtos.PrefixFilter.newBuilder();
|
||||
if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
|
||||
if (this.prefix != null) builder.setPrefix(ZeroCopyLiteralByteString.wrap(this.prefix));
|
||||
return builder.build().toByteArray();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -38,8 +38,9 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
|
||||
|
@ -306,10 +307,10 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
FilterProtos.SingleColumnValueFilter.Builder builder =
|
||||
FilterProtos.SingleColumnValueFilter.newBuilder();
|
||||
if (this.columnFamily != null) {
|
||||
builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
|
||||
builder.setColumnFamily(ZeroCopyLiteralByteString.wrap(this.columnFamily));
|
||||
}
|
||||
if (this.columnQualifier != null) {
|
||||
builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
|
||||
builder.setColumnQualifier(ZeroCopyLiteralByteString.wrap(this.columnQualifier));
|
||||
}
|
||||
HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
|
||||
builder.setCompareOp(compareOp);
|
||||
|
|
|
@ -18,9 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -30,7 +29,9 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
|
||||
import java.io.IOException;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
|
||||
|
@ -60,7 +61,7 @@ public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
|||
|
||||
final ClientProtos.CoprocessorServiceCall call =
|
||||
ClientProtos.CoprocessorServiceCall.newBuilder()
|
||||
.setRow(ByteString.copyFrom(HConstants.EMPTY_BYTE_ARRAY))
|
||||
.setRow(ZeroCopyLiteralByteString.wrap(HConstants.EMPTY_BYTE_ARRAY))
|
||||
.setServiceName(method.getService().getFullName())
|
||||
.setMethodName(method.getName())
|
||||
.setRequest(request.toByteString()).build();
|
||||
|
|
|
@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
|
||||
|
@ -76,7 +76,7 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
|||
|
||||
final ClientProtos.CoprocessorServiceCall call =
|
||||
ClientProtos.CoprocessorServiceCall.newBuilder()
|
||||
.setRow(ByteString.copyFrom(row))
|
||||
.setRow(ZeroCopyLiteralByteString.wrap(row))
|
||||
.setServiceName(method.getService().getFullName())
|
||||
.setMethodName(method.getName())
|
||||
.setRequest(request.toByteString()).build();
|
||||
|
|
|
@ -138,6 +138,7 @@ import com.google.protobuf.RpcChannel;
|
|||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Protobufs utility.
|
||||
|
@ -758,17 +759,17 @@ public final class ProtobufUtil {
|
|||
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
|
||||
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
|
||||
attributeBuilder.setName(attribute.getKey());
|
||||
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
|
||||
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
|
||||
scanBuilder.addAttribute(attributeBuilder.build());
|
||||
}
|
||||
}
|
||||
byte[] startRow = scan.getStartRow();
|
||||
if (startRow != null && startRow.length > 0) {
|
||||
scanBuilder.setStartRow(ByteString.copyFrom(startRow));
|
||||
scanBuilder.setStartRow(ZeroCopyLiteralByteString.wrap(startRow));
|
||||
}
|
||||
byte[] stopRow = scan.getStopRow();
|
||||
if (stopRow != null && stopRow.length > 0) {
|
||||
scanBuilder.setStopRow(ByteString.copyFrom(stopRow));
|
||||
scanBuilder.setStopRow(ZeroCopyLiteralByteString.wrap(stopRow));
|
||||
}
|
||||
if (scan.hasFilter()) {
|
||||
scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
|
||||
|
@ -777,12 +778,12 @@ public final class ProtobufUtil {
|
|||
Column.Builder columnBuilder = Column.newBuilder();
|
||||
for (Map.Entry<byte[],NavigableSet<byte []>>
|
||||
family: scan.getFamilyMap().entrySet()) {
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
|
||||
NavigableSet<byte []> qualifiers = family.getValue();
|
||||
columnBuilder.clearQualifier();
|
||||
if (qualifiers != null && qualifiers.size() > 0) {
|
||||
for (byte [] qualifier: qualifiers) {
|
||||
columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
|
||||
columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
}
|
||||
scanBuilder.addColumn(columnBuilder.build());
|
||||
|
@ -884,7 +885,7 @@ public final class ProtobufUtil {
|
|||
final Get get) throws IOException {
|
||||
ClientProtos.Get.Builder builder =
|
||||
ClientProtos.Get.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(get.getRow()));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(get.getRow()));
|
||||
builder.setCacheBlocks(get.getCacheBlocks());
|
||||
builder.setMaxVersions(get.getMaxVersions());
|
||||
if (get.getFilter() != null) {
|
||||
|
@ -903,7 +904,7 @@ public final class ProtobufUtil {
|
|||
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
|
||||
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
|
||||
attributeBuilder.setName(attribute.getKey());
|
||||
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
|
||||
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
|
||||
builder.addAttribute(attributeBuilder.build());
|
||||
}
|
||||
}
|
||||
|
@ -912,11 +913,11 @@ public final class ProtobufUtil {
|
|||
Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
|
||||
for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
|
||||
NavigableSet<byte[]> qualifiers = family.getValue();
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
|
||||
columnBuilder.clearQualifier();
|
||||
if (qualifiers != null && qualifiers.size() > 0) {
|
||||
for (byte[] qualifier: qualifiers) {
|
||||
columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
|
||||
columnBuilder.addQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
}
|
||||
builder.addColumn(columnBuilder.build());
|
||||
|
@ -945,7 +946,7 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static MutationProto toMutation(final Increment increment) {
|
||||
MutationProto.Builder builder = MutationProto.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(increment.getRow()));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow()));
|
||||
builder.setMutateType(MutationType.INCREMENT);
|
||||
builder.setDurability(toDurability(increment.getDurability()));
|
||||
TimeRange timeRange = increment.getTimeRange();
|
||||
|
@ -959,14 +960,14 @@ public final class ProtobufUtil {
|
|||
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
|
||||
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
|
||||
for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) {
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
|
||||
columnBuilder.clearQualifierValue();
|
||||
List<Cell> values = family.getValue();
|
||||
if (values != null && values.size() > 0) {
|
||||
for (Cell cell: values) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
|
||||
valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
|
||||
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(kv.getQualifier()));
|
||||
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(kv.getValue()));
|
||||
columnBuilder.addQualifierValue(valueBuilder.build());
|
||||
}
|
||||
}
|
||||
|
@ -989,12 +990,12 @@ public final class ProtobufUtil {
|
|||
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
|
||||
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
|
||||
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family.getKey()));
|
||||
columnBuilder.clearQualifierValue();
|
||||
for (Cell cell: family.getValue()) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
|
||||
valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
|
||||
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(kv.getQualifier()));
|
||||
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(kv.getValue()));
|
||||
valueBuilder.setTimestamp(kv.getTimestamp());
|
||||
if(cell.getTagsLength() > 0) {
|
||||
valueBuilder.setTags(ByteString.copyFrom(CellUtil.getTagArray(kv)));
|
||||
|
@ -1035,7 +1036,7 @@ public final class ProtobufUtil {
|
|||
private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
|
||||
final Mutation mutation) {
|
||||
MutationProto.Builder builder = MutationProto.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(mutation.getRow()));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(mutation.getRow()));
|
||||
builder.setMutateType(type);
|
||||
builder.setDurability(toDurability(mutation.getDurability()));
|
||||
builder.setTimestamp(mutation.getTimeStamp());
|
||||
|
@ -1044,7 +1045,7 @@ public final class ProtobufUtil {
|
|||
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
|
||||
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
|
||||
attributeBuilder.setName(attribute.getKey());
|
||||
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
|
||||
attributeBuilder.setValue(ZeroCopyLiteralByteString.wrap(attribute.getValue()));
|
||||
builder.addAttribute(attributeBuilder.build());
|
||||
}
|
||||
}
|
||||
|
@ -1159,7 +1160,7 @@ public final class ProtobufUtil {
|
|||
public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) {
|
||||
ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
|
||||
builder.setName(comparator.getClass().getName());
|
||||
builder.setSerializedComparator(ByteString.copyFrom(comparator.toByteArray()));
|
||||
builder.setSerializedComparator(ZeroCopyLiteralByteString.wrap(comparator.toByteArray()));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -1221,7 +1222,7 @@ public final class ProtobufUtil {
|
|||
public static FilterProtos.Filter toFilter(Filter filter) throws IOException {
|
||||
FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder();
|
||||
builder.setName(filter.getClass().getName());
|
||||
builder.setSerializedFilter(ByteString.copyFrom(filter.toByteArray()));
|
||||
builder.setSerializedFilter(ZeroCopyLiteralByteString.wrap(filter.toByteArray()));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -1706,10 +1707,10 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.TablePermission.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
|
||||
if (tablePerm.hasFamily()) {
|
||||
builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
|
||||
builder.setFamily(ZeroCopyLiteralByteString.wrap(tablePerm.getFamily()));
|
||||
}
|
||||
if (tablePerm.hasQualifier()) {
|
||||
builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
|
||||
builder.setQualifier(ZeroCopyLiteralByteString.wrap(tablePerm.getQualifier()));
|
||||
}
|
||||
for (Permission.Action a : perm.getActions()) {
|
||||
builder.addAction(toPermissionAction(a));
|
||||
|
@ -1798,7 +1799,7 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
|
||||
return AccessControlProtos.UserPermission.newBuilder()
|
||||
.setUser(ByteString.copyFrom(perm.getUser()))
|
||||
.setUser(ZeroCopyLiteralByteString.wrap(perm.getUser()))
|
||||
.setPermission(toPermission(perm))
|
||||
.build();
|
||||
}
|
||||
|
@ -2070,8 +2071,8 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
|
||||
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
|
||||
builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
|
||||
builder.setPassword(ByteString.copyFrom(token.getPassword()));
|
||||
builder.setIdentifier(ZeroCopyLiteralByteString.wrap(token.getIdentifier()));
|
||||
builder.setPassword(ZeroCopyLiteralByteString.wrap(token.getPassword()));
|
||||
if (token.getService() != null) {
|
||||
builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
|
||||
}
|
||||
|
@ -2254,9 +2255,9 @@ public final class ProtobufUtil {
|
|||
// input / output paths are relative to the store dir
|
||||
// store dir is relative to region dir
|
||||
CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
|
||||
.setTableName(ByteString.copyFrom(info.getTableName()))
|
||||
.setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
|
||||
.setFamilyName(ByteString.copyFrom(family))
|
||||
.setTableName(ZeroCopyLiteralByteString.wrap(info.getTableName()))
|
||||
.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(info.getEncodedNameAsBytes()))
|
||||
.setFamilyName(ZeroCopyLiteralByteString.wrap(family))
|
||||
.setStoreHomeDir(storeDir.getName()); //make relative
|
||||
for (Path inputPath : inputPaths) {
|
||||
builder.addCompactionInput(inputPath.getName()); //relative path
|
||||
|
@ -2333,8 +2334,8 @@ public final class ProtobufUtil {
|
|||
|
||||
public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
|
||||
return HBaseProtos.TableName.newBuilder()
|
||||
.setNamespace(ByteString.copyFrom(tableName.getNamespace()))
|
||||
.setQualifier(ByteString.copyFrom(tableName.getQualifier())).build();
|
||||
.setNamespace(ZeroCopyLiteralByteString.wrap(tableName.getNamespace()))
|
||||
.setQualifier(ZeroCopyLiteralByteString.wrap(tableName.getQualifier())).build();
|
||||
}
|
||||
|
||||
public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {
|
||||
|
|
|
@ -22,15 +22,16 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Action;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
|
@ -38,7 +39,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
|
@ -99,9 +99,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLa
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Triple;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Helper utility to build protocol buffer requests,
|
||||
|
@ -133,10 +133,10 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
|
||||
Column.Builder columnBuilder = Column.newBuilder();
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
ClientProtos.Get.Builder getBuilder =
|
||||
ClientProtos.Get.newBuilder();
|
||||
getBuilder.setRow(ByteString.copyFrom(row));
|
||||
getBuilder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
getBuilder.addColumn(columnBuilder.build());
|
||||
getBuilder.setClosestRowBefore(true);
|
||||
builder.setGet(getBuilder.build());
|
||||
|
@ -181,14 +181,14 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFrom(row));
|
||||
mutateBuilder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
mutateBuilder.setMutateType(MutationType.INCREMENT);
|
||||
mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
|
||||
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family));
|
||||
columnBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
|
||||
valueBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(amount)));
|
||||
valueBuilder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
valueBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(amount)));
|
||||
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
columnBuilder.addQualifierValue(valueBuilder.build());
|
||||
mutateBuilder.addColumnValue(columnBuilder.build());
|
||||
builder.setMutation(mutateBuilder.build());
|
||||
|
@ -466,7 +466,7 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder();
|
||||
for (Pair<byte[], String> familyPath: familyPaths) {
|
||||
familyPathBuilder.setFamily(ByteString.copyFrom(familyPath.getFirst()));
|
||||
familyPathBuilder.setFamily(ZeroCopyLiteralByteString.wrap(familyPath.getFirst()));
|
||||
familyPathBuilder.setPath(familyPath.getSecond());
|
||||
builder.addFamilyPath(familyPathBuilder.build());
|
||||
}
|
||||
|
@ -629,7 +629,7 @@ public final class RequestConverter {
|
|||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.addFamily(ByteString.copyFrom(family));
|
||||
builder.addFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -775,7 +775,7 @@ public final class RequestConverter {
|
|||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
if (splitPoint != null) {
|
||||
builder.setSplitPoint(ByteString.copyFrom(splitPoint));
|
||||
builder.setSplitPoint(ZeroCopyLiteralByteString.wrap(splitPoint));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -815,7 +815,7 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
builder.setMajor(major);
|
||||
if (family != null) {
|
||||
builder.setFamily(ByteString.copyFrom(family));
|
||||
builder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -874,7 +874,7 @@ public final class RequestConverter {
|
|||
public static RegionSpecifier buildRegionSpecifier(
|
||||
final RegionSpecifierType type, final byte[] value) {
|
||||
RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder();
|
||||
regionBuilder.setValue(ByteString.copyFrom(value));
|
||||
regionBuilder.setValue(ZeroCopyLiteralByteString.wrap(value));
|
||||
regionBuilder.setType(type);
|
||||
return regionBuilder.build();
|
||||
}
|
||||
|
@ -895,9 +895,9 @@ public final class RequestConverter {
|
|||
final ByteArrayComparable comparator,
|
||||
final CompareType compareType) throws IOException {
|
||||
Condition.Builder builder = Condition.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(row));
|
||||
builder.setFamily(ByteString.copyFrom(family));
|
||||
builder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
builder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
builder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
builder.setComparator(ProtobufUtil.toComparator(comparator));
|
||||
builder.setCompareType(compareType);
|
||||
return builder.build();
|
||||
|
@ -929,7 +929,7 @@ public final class RequestConverter {
|
|||
final TableName tableName, final byte [] columnName) {
|
||||
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
|
||||
builder.setColumnName(ByteString.copyFrom(columnName));
|
||||
builder.setColumnName(ZeroCopyLiteralByteString.wrap(columnName));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -1069,7 +1069,7 @@ public final class RequestConverter {
|
|||
builder.setTableSchema(hTableDesc.convert());
|
||||
if (splitKeys != null) {
|
||||
for (byte [] splitKey : splitKeys) {
|
||||
builder.addSplitKeys(ByteString.copyFrom(splitKey));
|
||||
builder.addSplitKeys(ZeroCopyLiteralByteString.wrap(splitKey));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
|
@ -1222,7 +1222,7 @@ public final class RequestConverter {
|
|||
public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest(
|
||||
byte[] regionName) {
|
||||
return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName(
|
||||
ByteString.copyFrom(regionName)).build();
|
||||
ZeroCopyLiteralByteString.wrap(regionName)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1277,10 +1277,10 @@ public final class RequestConverter {
|
|||
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
|
||||
if (family != null) {
|
||||
permissionBuilder.setFamily(ByteString.copyFrom(family));
|
||||
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
}
|
||||
if (qualifier != null) {
|
||||
permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
ret.setType(AccessControlProtos.Permission.Type.Table)
|
||||
.setTablePermission(permissionBuilder);
|
||||
|
@ -1373,10 +1373,10 @@ public final class RequestConverter {
|
|||
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
}
|
||||
if (family != null) {
|
||||
permissionBuilder.setFamily(ByteString.copyFrom(family));
|
||||
permissionBuilder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
}
|
||||
if (qualifier != null) {
|
||||
permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
permissionBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
ret.setType(AccessControlProtos.Permission.Type.Table)
|
||||
.setTablePermission(permissionBuilder);
|
||||
|
|
|
@ -69,8 +69,8 @@ import org.apache.zookeeper.proto.DeleteRequest;
|
|||
import org.apache.zookeeper.proto.SetDataRequest;
|
||||
import org.apache.zookeeper.server.ZooKeeperSaslServer;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Internal HBase utility class for ZooKeeper.
|
||||
|
@ -1946,7 +1946,7 @@ public class ZKUtil {
|
|||
if (storeSequenceIds != null) {
|
||||
for (byte[] columnFamilyName : storeSequenceIds.keySet()) {
|
||||
Long curSeqId = storeSequenceIds.get(columnFamilyName);
|
||||
storeSequenceIdBuilder.setFamilyName(ByteString.copyFrom(columnFamilyName));
|
||||
storeSequenceIdBuilder.setFamilyName(ZeroCopyLiteralByteString.wrap(columnFamilyName));
|
||||
storeSequenceIdBuilder.setSequenceId(curSeqId);
|
||||
regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
|
||||
storeSequenceIdBuilder.clear();
|
||||
|
|
|
@ -0,0 +1,56 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package com.google.protobuf; // This is a lie.
|
||||
|
||||
/**
|
||||
* Helper class to extract byte arrays from {@link ByteString} without copy.
|
||||
* <p>
|
||||
* Without this protobufs would force us to copy every single byte array out
|
||||
* of the objects de-serialized from the wire (which already do one copy, on
|
||||
* top of the copies the JVM does to go from kernel buffer to C buffer and
|
||||
* from C buffer to JVM buffer).
|
||||
*
|
||||
* @since 0.96.1
|
||||
*/
|
||||
public final class ZeroCopyLiteralByteString extends LiteralByteString {
|
||||
// Gotten from AsyncHBase code base with permission.
|
||||
/** Private constructor so this class cannot be instantiated. */
|
||||
private ZeroCopyLiteralByteString() {
|
||||
super(null);
|
||||
throw new UnsupportedOperationException("Should never be here.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a byte array in a {@link ByteString} without copying it.
|
||||
*/
|
||||
public static ByteString wrap(final byte[] array) {
|
||||
return new LiteralByteString(array);
|
||||
}
|
||||
|
||||
// TODO:
|
||||
// ZeroCopyLiteralByteString.wrap(this.buf, 0, this.count);
|
||||
|
||||
/**
|
||||
* Extracts the byte array from the given {@link ByteString} without copy.
|
||||
* @param buf A buffer from which to extract the array. This buffer must be
|
||||
* actually an instance of a {@code LiteralByteString}.
|
||||
*/
|
||||
public static byte[] zeroCopyGetBytes(final LiteralByteString buf) {
|
||||
return buf.bytes;
|
||||
}
|
||||
}
|
|
@ -24,13 +24,9 @@ import java.io.OutputStream;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.codec.BaseDecoder;
|
||||
import org.apache.hadoop.hbase.codec.BaseEncoder;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp.
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* A reference to the top or bottom half of a store file where 'bottom' is the first half
|
||||
|
@ -194,7 +195,7 @@ public class Reference {
|
|||
FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
|
||||
builder.setRange(isTopFileRegion(getFileRegion())?
|
||||
FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
|
||||
builder.setSplitkey(ByteString.copyFrom(getSplitKey()));
|
||||
builder.setSplitkey(ZeroCopyLiteralByteString.wrap(getSplitKey()));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.ChecksumType;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -191,18 +192,18 @@ public class HFile {
|
|||
static final AtomicLong checksumFailures = new AtomicLong();
|
||||
|
||||
// For getting more detailed stats on FS latencies
|
||||
// If, for some reason, the metrics subsystem stops polling for latencies,
|
||||
// If, for some reason, the metrics subsystem stops polling for latencies,
|
||||
// I don't want data to pile up in a memory leak
|
||||
// so, after LATENCY_BUFFER_SIZE items have been enqueued for processing,
|
||||
// fs latency stats will be dropped (and this behavior will be logged)
|
||||
private static final int LATENCY_BUFFER_SIZE = 5000;
|
||||
private static final BlockingQueue<Long> fsReadLatenciesNanos =
|
||||
private static final BlockingQueue<Long> fsReadLatenciesNanos =
|
||||
new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
|
||||
private static final BlockingQueue<Long> fsWriteLatenciesNanos =
|
||||
private static final BlockingQueue<Long> fsWriteLatenciesNanos =
|
||||
new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
|
||||
private static final BlockingQueue<Long> fsPreadLatenciesNanos =
|
||||
private static final BlockingQueue<Long> fsPreadLatenciesNanos =
|
||||
new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
|
||||
|
||||
|
||||
public static final void offerReadLatency(long latencyNanos, boolean pread) {
|
||||
if (pread) {
|
||||
fsPreadLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
|
||||
|
@ -214,30 +215,30 @@ public class HFile {
|
|||
readOps.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static final void offerWriteLatency(long latencyNanos) {
|
||||
fsWriteLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
|
||||
|
||||
|
||||
writeTimeNano.addAndGet(latencyNanos);
|
||||
writeOps.incrementAndGet();
|
||||
}
|
||||
|
||||
|
||||
public static final Collection<Long> getReadLatenciesNanos() {
|
||||
final List<Long> latencies =
|
||||
final List<Long> latencies =
|
||||
Lists.newArrayListWithCapacity(fsReadLatenciesNanos.size());
|
||||
fsReadLatenciesNanos.drainTo(latencies);
|
||||
return latencies;
|
||||
}
|
||||
|
||||
public static final Collection<Long> getPreadLatenciesNanos() {
|
||||
final List<Long> latencies =
|
||||
final List<Long> latencies =
|
||||
Lists.newArrayListWithCapacity(fsPreadLatenciesNanos.size());
|
||||
fsPreadLatenciesNanos.drainTo(latencies);
|
||||
return latencies;
|
||||
}
|
||||
|
||||
|
||||
public static final Collection<Long> getWriteLatenciesNanos() {
|
||||
final List<Long> latencies =
|
||||
final List<Long> latencies =
|
||||
Lists.newArrayListWithCapacity(fsWriteLatenciesNanos.size());
|
||||
fsWriteLatenciesNanos.drainTo(latencies);
|
||||
return latencies;
|
||||
|
@ -380,7 +381,7 @@ public class HFile {
|
|||
if (path != null) {
|
||||
ostream = AbstractHFileWriter.createOutputStream(conf, fs, path, favoredNodes);
|
||||
}
|
||||
return createWriter(fs, path, ostream,
|
||||
return createWriter(fs, path, ostream,
|
||||
comparator, fileContext);
|
||||
}
|
||||
|
||||
|
@ -743,8 +744,8 @@ public class HFile {
|
|||
HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
|
||||
for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
|
||||
HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
|
||||
bbpBuilder.setFirst(ByteString.copyFrom(e.getKey()));
|
||||
bbpBuilder.setSecond(ByteString.copyFrom(e.getValue()));
|
||||
bbpBuilder.setFirst(ZeroCopyLiteralByteString.wrap(e.getKey()));
|
||||
bbpBuilder.setSecond(ZeroCopyLiteralByteString.wrap(e.getValue()));
|
||||
builder.addMapEntry(bbpBuilder.build());
|
||||
}
|
||||
out.write(ProtobufUtil.PB_MAGIC);
|
||||
|
@ -786,7 +787,7 @@ public class HFile {
|
|||
/** Now parse the old Writable format. It was a list of Map entries. Each map entry was a key and a value of
|
||||
* a byte []. The old map format had a byte before each entry that held a code which was short for the key or
|
||||
* value type. We know it was a byte [] so in below we just read and dump it.
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
void parseWritable(final DataInputStream in) throws IOException {
|
||||
// First clear the map. Otherwise we will just accumulate entries every time this method is called.
|
||||
|
|
|
@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ReplicationProtbufUtil {
|
||||
|
@ -89,8 +89,8 @@ public class ReplicationProtbufUtil {
|
|||
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
|
||||
HLogKey key = entry.getKey();
|
||||
keyBuilder.setEncodedRegionName(
|
||||
ByteString.copyFrom(key.getEncodedRegionName()));
|
||||
keyBuilder.setTableName(ByteString.copyFrom(key.getTablename().getName()));
|
||||
ZeroCopyLiteralByteString.wrap(key.getEncodedRegionName()));
|
||||
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName()));
|
||||
keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
|
||||
keyBuilder.setWriteTime(key.getWriteTime());
|
||||
for(UUID clusterId : key.getClusterIds()) {
|
||||
|
@ -102,7 +102,7 @@ public class ReplicationProtbufUtil {
|
|||
NavigableMap<byte[], Integer> scopes = key.getScopes();
|
||||
if (scopes != null && !scopes.isEmpty()) {
|
||||
for (Map.Entry<byte[], Integer> scope: scopes.entrySet()) {
|
||||
scopeBuilder.setFamily(ByteString.copyFrom(scope.getKey()));
|
||||
scopeBuilder.setFamily(ZeroCopyLiteralByteString.wrap(scope.getKey()));
|
||||
WALProtos.ScopeType scopeType =
|
||||
WALProtos.ScopeType.valueOf(scope.getValue().intValue());
|
||||
scopeBuilder.setScopeType(scopeType);
|
||||
|
|
|
@ -197,7 +197,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -235,6 +234,7 @@ import com.google.protobuf.Message;
|
|||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* HRegionServer makes a set of HRegions available to clients. It checks in with
|
||||
|
@ -1296,7 +1296,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
RegionLoad.Builder regionLoad = RegionLoad.newBuilder();
|
||||
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
||||
regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
|
||||
regionSpecifier.setValue(ByteString.copyFrom(name));
|
||||
regionSpecifier.setValue(ZeroCopyLiteralByteString.wrap(name));
|
||||
regionLoad.setRegionSpecifier(regionSpecifier.build())
|
||||
.setStores(stores)
|
||||
.setStorefiles(storefiles)
|
||||
|
@ -3920,7 +3920,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
|
||||
if (regionsToFlush != null) {
|
||||
for (byte[] region: regionsToFlush) {
|
||||
builder.addRegionToFlush(ByteString.copyFrom(region));
|
||||
builder.addRegionToFlush(ZeroCopyLiteralByteString.wrap(region));
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
|
@ -35,9 +34,9 @@ import java.util.UUID;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
|
||||
|
@ -47,6 +46,7 @@ import org.apache.hadoop.io.WritableComparable;
|
|||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* A Key for an entry in the change log.
|
||||
|
@ -425,8 +425,8 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
WALCellCodec.ByteStringCompressor compressor) throws IOException {
|
||||
WALKey.Builder builder = WALKey.newBuilder();
|
||||
if (compressionContext == null) {
|
||||
builder.setEncodedRegionName(ByteString.copyFrom(this.encodedRegionName));
|
||||
builder.setTableName(ByteString.copyFrom(this.tablename.getName()));
|
||||
builder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(this.encodedRegionName));
|
||||
builder.setTableName(ZeroCopyLiteralByteString.wrap(this.tablename.getName()));
|
||||
} else {
|
||||
builder.setEncodedRegionName(
|
||||
compressor.compress(this.encodedRegionName, compressionContext.regionDict));
|
||||
|
@ -443,7 +443,7 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
}
|
||||
if (scopes != null) {
|
||||
for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
|
||||
ByteString family = (compressionContext == null) ? ByteString.copyFrom(e.getKey())
|
||||
ByteString family = (compressionContext == null) ? ZeroCopyLiteralByteString.wrap(e.getKey())
|
||||
: compressor.compress(e.getKey(), compressionContext.familyDict);
|
||||
builder.addScopes(FamilyScope.newBuilder()
|
||||
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));
|
||||
|
|
|
@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
|
||||
import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Representation of a cell. A cell is a single value associated a column and
|
||||
* optional qualifier, and either the timestamp when it was stored or the user-
|
||||
|
@ -185,8 +185,8 @@ public class CellModel implements ProtobufMessageHandler, Serializable {
|
|||
@Override
|
||||
public byte[] createProtobufOutput() {
|
||||
Cell.Builder builder = Cell.newBuilder();
|
||||
builder.setColumn(ByteString.copyFrom(getColumn()));
|
||||
builder.setData(ByteString.copyFrom(getValue()));
|
||||
builder.setColumn(ZeroCopyLiteralByteString.wrap(getColumn()));
|
||||
builder.setData(ZeroCopyLiteralByteString.wrap(getValue()));
|
||||
if (hasUserTimestamp()) {
|
||||
builder.setTimestamp(getTimestamp());
|
||||
}
|
||||
|
|
|
@ -26,8 +26,8 @@ import java.util.List;
|
|||
|
||||
import javax.xml.bind.annotation.XmlAccessType;
|
||||
import javax.xml.bind.annotation.XmlAccessorType;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import javax.xml.bind.annotation.XmlElement;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -35,7 +35,7 @@ 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.CellSetMessage.CellSet;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Representation of a grouping of cells. May contain cells from more than
|
||||
|
@ -115,11 +115,11 @@ public class CellSetModel implements Serializable, ProtobufMessageHandler {
|
|||
CellSet.Builder builder = CellSet.newBuilder();
|
||||
for (RowModel row: getRows()) {
|
||||
CellSet.Row.Builder rowBuilder = CellSet.Row.newBuilder();
|
||||
rowBuilder.setKey(ByteString.copyFrom(row.getKey()));
|
||||
rowBuilder.setKey(ZeroCopyLiteralByteString.wrap(row.getKey()));
|
||||
for (CellModel cell: row.getCells()) {
|
||||
Cell.Builder cellBuilder = Cell.newBuilder();
|
||||
cellBuilder.setColumn(ByteString.copyFrom(cell.getColumn()));
|
||||
cellBuilder.setData(ByteString.copyFrom(cell.getValue()));
|
||||
cellBuilder.setColumn(ZeroCopyLiteralByteString.wrap(cell.getColumn()));
|
||||
cellBuilder.setData(ZeroCopyLiteralByteString.wrap(cell.getValue()));
|
||||
if (cell.hasUserTimestamp()) {
|
||||
cellBuilder.setTimestamp(cell.getTimestamp());
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.util.Base64;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
import com.sun.jersey.api.json.JSONConfiguration;
|
||||
import com.sun.jersey.api.json.JSONJAXBContext;
|
||||
import com.sun.jersey.api.json.JSONMarshaller;
|
||||
|
@ -708,13 +708,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
|
|||
public byte[] createProtobufOutput() {
|
||||
Scanner.Builder builder = Scanner.newBuilder();
|
||||
if (!Bytes.equals(startRow, HConstants.EMPTY_START_ROW)) {
|
||||
builder.setStartRow(ByteString.copyFrom(startRow));
|
||||
builder.setStartRow(ZeroCopyLiteralByteString.wrap(startRow));
|
||||
}
|
||||
if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) {
|
||||
builder.setEndRow(ByteString.copyFrom(endRow));
|
||||
builder.setEndRow(ZeroCopyLiteralByteString.wrap(endRow));
|
||||
}
|
||||
for (byte[] column: columns) {
|
||||
builder.addColumns(ByteString.copyFrom(column));
|
||||
builder.addColumns(ZeroCopyLiteralByteString.wrap(column));
|
||||
}
|
||||
if (startTime != 0) {
|
||||
builder.setStartTime(startTime);
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
|
|||
import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Representation of the status of a storage cluster:
|
||||
|
@ -722,7 +722,7 @@ public class StorageClusterStatusModel
|
|||
for (Node.Region region: node.regions) {
|
||||
StorageClusterStatus.Region.Builder regionBuilder =
|
||||
StorageClusterStatus.Region.newBuilder();
|
||||
regionBuilder.setName(ByteString.copyFrom(region.name));
|
||||
regionBuilder.setName(ZeroCopyLiteralByteString.wrap(region.name));
|
||||
regionBuilder.setStores(region.stores);
|
||||
regionBuilder.setStorefiles(region.storefiles);
|
||||
regionBuilder.setStorefileSizeMB(region.storefileSizeMB);
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
|
||||
import org.apache.hadoop.hbase.rest.protobuf.generated.TableInfoMessage.TableInfo;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Representation of a list of table regions.
|
||||
|
@ -135,8 +135,8 @@ public class TableInfoModel implements Serializable, ProtobufMessageHandler {
|
|||
TableInfo.Region.Builder regionBuilder = TableInfo.Region.newBuilder();
|
||||
regionBuilder.setName(aRegion.getName());
|
||||
regionBuilder.setId(aRegion.getId());
|
||||
regionBuilder.setStartKey(ByteString.copyFrom(aRegion.getStartKey()));
|
||||
regionBuilder.setEndKey(ByteString.copyFrom(aRegion.getEndKey()));
|
||||
regionBuilder.setStartKey(ZeroCopyLiteralByteString.wrap(aRegion.getStartKey()));
|
||||
regionBuilder.setEndKey(ZeroCopyLiteralByteString.wrap(aRegion.getEndKey()));
|
||||
regionBuilder.setLocation(aRegion.getLocation());
|
||||
builder.addRegions(regionBuilder);
|
||||
}
|
||||
|
|
|
@ -18,18 +18,21 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -37,6 +40,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -53,11 +57,9 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* TestEndpoint: test cases to verify coprocessor Endpoint
|
||||
|
@ -123,9 +125,9 @@ public class TestCoprocessorEndpoint {
|
|||
new BlockingRpcCallback<ColumnAggregationProtos.SumResponse>();
|
||||
ColumnAggregationProtos.SumRequest.Builder builder =
|
||||
ColumnAggregationProtos.SumRequest.newBuilder();
|
||||
builder.setFamily(ByteString.copyFrom(family));
|
||||
builder.setFamily(ZeroCopyLiteralByteString.wrap(family));
|
||||
if (qualifier != null && qualifier.length > 0) {
|
||||
builder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
builder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
|
||||
}
|
||||
instance.sum(null, builder.build(), rpcCallback);
|
||||
return rpcCallback.get().getSum();
|
||||
|
|
|
@ -32,46 +32,45 @@ import java.util.Set;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.IsolationLevel;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
|
||||
import org.apache.hadoop.hbase.regionserver.BaseRowProcessor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
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.LogFactory;
|
||||
|
||||
|
@ -363,7 +362,7 @@ public class TestRowProcessorEndpoint {
|
|||
public IncCounterProcessorRequest getRequestData() throws IOException {
|
||||
IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
|
||||
builder.setCounter(counter);
|
||||
builder.setRow(ByteString.copyFrom(row));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -442,8 +441,8 @@ public class TestRowProcessorEndpoint {
|
|||
public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
|
||||
FriendsOfFriendsProcessorRequest.Builder builder =
|
||||
FriendsOfFriendsProcessorRequest.newBuilder();
|
||||
builder.setPerson(ByteString.copyFrom(person));
|
||||
builder.setRow(ByteString.copyFrom(row));
|
||||
builder.setPerson(ZeroCopyLiteralByteString.wrap(person));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
builder.addAllResult(result);
|
||||
FriendsOfFriendsProcessorRequest f = builder.build();
|
||||
return f;
|
||||
|
@ -547,8 +546,8 @@ public class TestRowProcessorEndpoint {
|
|||
@Override
|
||||
public RowSwapProcessorRequest getRequestData() throws IOException {
|
||||
RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
|
||||
builder.setRow1(ByteString.copyFrom(row1));
|
||||
builder.setRow2(ByteString.copyFrom(row2));
|
||||
builder.setRow1(ZeroCopyLiteralByteString.wrap(row1));
|
||||
builder.setRow2(ZeroCopyLiteralByteString.wrap(row2));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -607,7 +606,7 @@ public class TestRowProcessorEndpoint {
|
|||
@Override
|
||||
public TimeoutProcessorRequest getRequestData() throws IOException {
|
||||
TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(row));
|
||||
builder.setRow(ZeroCopyLiteralByteString.wrap(row));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
/**
|
||||
* Class to test ProtobufUtil.
|
||||
|
@ -53,12 +54,12 @@ public class TestProtobufUtil {
|
|||
NameBytesPair.Builder builder = NameBytesPair.newBuilder();
|
||||
final String omg = "OMG!!!";
|
||||
builder.setName("java.io.IOException");
|
||||
builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
|
||||
builder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(omg)));
|
||||
Throwable t = ProtobufUtil.toException(builder.build());
|
||||
assertEquals(omg, t.getMessage());
|
||||
builder.clear();
|
||||
builder.setName("org.apache.hadoop.ipc.RemoteException");
|
||||
builder.setValue(ByteString.copyFrom(Bytes.toBytes(omg)));
|
||||
builder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(omg)));
|
||||
t = ProtobufUtil.toException(builder.build());
|
||||
assertEquals(omg, t.getMessage());
|
||||
}
|
||||
|
@ -202,10 +203,10 @@ public class TestProtobufUtil {
|
|||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
|
||||
qualifierBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(11L)));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
|
||||
qualifierBuilder.setValue(ZeroCopyLiteralByteString.wrap(Bytes.toBytes(22L)));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.junit.experimental.categories.Category;
|
|||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
/**
|
||||
* Tests that verify certain RPCs get a higher QoS.
|
||||
*/
|
||||
|
@ -69,12 +70,12 @@ public class TestPriorityRpc {
|
|||
GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
|
||||
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
|
||||
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
|
||||
ByteString name = ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
ByteString name = ZeroCopyLiteralByteString.wrap(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
regionSpecifierBuilder.setValue(name);
|
||||
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
|
||||
getRequestBuilder.setRegion(regionSpecifier);
|
||||
Get.Builder getBuilder = Get.newBuilder();
|
||||
getBuilder.setRow(ByteString.copyFrom("somerow".getBytes()));
|
||||
getBuilder.setRow(ZeroCopyLiteralByteString.wrap("somerow".getBytes()));
|
||||
getRequestBuilder.setGet(getBuilder.build());
|
||||
GetRequest getRequest = getRequestBuilder.build();
|
||||
RequestHeader header = headerBuilder.build();
|
||||
|
@ -143,4 +144,4 @@ public class TestPriorityRpc {
|
|||
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
|
||||
assertEquals(HConstants.NORMAL_QOS, priority.getPriority(header, scanRequest));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,13 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -44,7 +50,7 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestReplicationSink {
|
||||
|
@ -256,9 +262,9 @@ public class TestReplicationSink {
|
|||
uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
|
||||
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
|
||||
keyBuilder.setClusterId(uuidBuilder.build());
|
||||
keyBuilder.setTableName(ByteString.copyFrom(table));
|
||||
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(table));
|
||||
keyBuilder.setWriteTime(now);
|
||||
keyBuilder.setEncodedRegionName(ByteString.copyFrom(HConstants.EMPTY_BYTE_ARRAY));
|
||||
keyBuilder.setEncodedRegionName(ZeroCopyLiteralByteString.wrap(HConstants.EMPTY_BYTE_ARRAY));
|
||||
keyBuilder.setLogSequenceNumber(-1);
|
||||
builder.setKey(keyBuilder.build());
|
||||
cells.add(kv);
|
||||
|
|
Loading…
Reference in New Issue