HBASE-11118 non environment variable solution for "IllegalAccessError: class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString"

This commit is contained in:
Nick Dimiduk 2014-07-07 19:04:53 -07:00
parent c6ddc0336e
commit c01e7e50c1
52 changed files with 298 additions and 237 deletions

View File

@ -26,7 +26,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.master.RegionState;
@ -334,7 +334,7 @@ public class ClusterStatus extends VersionedWritable {
ClusterStatusProtos.RegionState rs = rit.getValue().convert();
RegionSpecifier.Builder spec =
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
spec.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(rit.getKey())));
spec.setValue(ByteStringer.wrap(Bytes.toBytes(rit.getKey())));
RegionInTransition pbRIT =
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparable;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -1299,11 +1299,11 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
*/
public ColumnFamilySchema convert() {
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
builder.setName(HBaseZeroCopyByteString.wrap(getName()));
builder.setName(ByteStringer.wrap(getName()));
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(HBaseZeroCopyByteString.wrap(e.getKey().get()));
aBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue().get()));
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
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.List;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -48,7 +49,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.io.DataInputBuffer;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -993,10 +993,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) {
builder.setStartKey(HBaseZeroCopyByteString.wrap(info.getStartKey()));
builder.setStartKey(ByteStringer.wrap(info.getStartKey()));
}
if (info.getEndKey() != null) {
builder.setEndKey(HBaseZeroCopyByteString.wrap(info.getEndKey()));
builder.setEndKey(ByteStringer.wrap(info.getEndKey()));
}
builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit());

View File

@ -34,6 +34,7 @@ import java.util.TreeMap;
import java.util.TreeSet;
import java.util.regex.Matcher;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -53,7 +54,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.WritableComparable;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -1448,8 +1448,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(HBaseZeroCopyByteString.wrap(e.getKey().get()));
aBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue().get()));
aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (HColumnDescriptor hcd: getColumnFamilies()) {

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
@ -104,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(HBaseZeroCopyByteString.wrap(regionName)).
setEventTypeCode(type.getCode()).setRegionName(ByteStringer.wrap(regionName)).
setServerName(pbsn);
builder.setCreateTime(System.currentTimeMillis());
if (payload != null) builder.setPayload(HBaseZeroCopyByteString.wrap(payload));
if (payload != null) builder.setPayload(ByteStringer.wrap(payload));
return new RegionTransition(builder.build());
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import java.math.BigDecimal;
import java.math.RoundingMode;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
@ -121,7 +121,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
private BigDecimalMsg getProtoForType(BigDecimal t) {
BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder();
return builder.setBigdecimalMsg(HBaseZeroCopyByteString.wrap(Bytes.toBytes(t))).build();
return builder.setBigdecimalMsg(ByteStringer.wrap(Bytes.toBytes(t))).build();
}
@Override

View File

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

View File

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

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
@ -29,7 +30,6 @@ 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.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -175,7 +175,7 @@ public class ColumnPaginationFilter extends FilterBase
builder.setOffset(this.offset);
}
if (this.columnOffset != null) {
builder.setColumnOffset(HBaseZeroCopyByteString.wrap(this.columnOffset));
builder.setColumnOffset(ByteStringer.wrap(this.columnOffset));
}
return builder.build().toByteArray();
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
@ -30,7 +31,6 @@ 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.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -95,7 +95,7 @@ public class ColumnPrefixFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.ColumnPrefixFilter.Builder builder =
FilterProtos.ColumnPrefixFilter.newBuilder();
if (this.prefix != null) builder.setPrefix(HBaseZeroCopyByteString.wrap(this.prefix));
if (this.prefix != null) builder.setPrefix(ByteStringer.wrap(this.prefix));
return builder.build().toByteArray();
}

View File

@ -21,7 +21,9 @@ package org.apache.hadoop.hbase.filter;
import static org.apache.hadoop.hbase.util.Bytes.len;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -31,9 +33,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
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.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.ArrayList;
/**
* This filter is used for selecting only those keys with columns that are
@ -173,9 +173,9 @@ public class ColumnRangeFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.ColumnRangeFilter.Builder builder =
FilterProtos.ColumnRangeFilter.newBuilder();
if (this.minColumn != null) builder.setMinColumn(HBaseZeroCopyByteString.wrap(this.minColumn));
if (this.minColumn != null) builder.setMinColumn(ByteStringer.wrap(this.minColumn));
builder.setMinColumnInclusive(this.minColumnInclusive);
if (this.maxColumn != null) builder.setMaxColumn(HBaseZeroCopyByteString.wrap(this.maxColumn));
if (this.maxColumn != null) builder.setMaxColumn(ByteStringer.wrap(this.maxColumn));
builder.setMaxColumnInclusive(this.maxColumnInclusive);
return builder.build().toByteArray();
}

View File

@ -25,6 +25,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
@ -37,7 +38,6 @@ 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.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -226,10 +226,10 @@ public class DependentColumnFilter extends CompareFilter {
FilterProtos.DependentColumnFilter.newBuilder();
builder.setCompareFilter(super.convert());
if (this.columnFamily != null) {
builder.setColumnFamily(HBaseZeroCopyByteString.wrap(this.columnFamily));
builder.setColumnFamily(ByteStringer.wrap(this.columnFamily));
}
if (this.columnQualifier != null) {
builder.setColumnQualifier(HBaseZeroCopyByteString.wrap(this.columnQualifier));
builder.setColumnQualifier(ByteStringer.wrap(this.columnQualifier));
}
builder.setDropDependentColumn(this.dropDependentColumn);
return builder.build().toByteArray();

View File

@ -18,21 +18,21 @@
package org.apache.hadoop.hbase.filter;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.Set;
import java.util.TreeSet;
/**
* The filter looks for the given columns in KeyValue. Once there is a match for
* any one of the columns, it returns ReturnCode.NEXT_ROW for remaining
@ -86,7 +86,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
for (byte[] qualifier : qualifiers) {
if (qualifier != null) builder.addQualifiers(HBaseZeroCopyByteString.wrap(qualifier));
if (qualifier != null) builder.addQualifiers(ByteStringer.wrap(qualifier));
}
return builder.build().toByteArray();
}

View File

@ -17,10 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
@ -29,11 +26,13 @@ import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Filters data based on fuzzy row key. Performs fast-forwards during scanning.
@ -147,8 +146,8 @@ public class FuzzyRowFilter extends FilterBase {
FilterProtos.FuzzyRowFilter.newBuilder();
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
bbpBuilder.setFirst(HBaseZeroCopyByteString.wrap(fuzzyData.getFirst()));
bbpBuilder.setSecond(HBaseZeroCopyByteString.wrap(fuzzyData.getSecond()));
bbpBuilder.setFirst(ByteStringer.wrap(fuzzyData.getFirst()));
bbpBuilder.setSecond(ByteStringer.wrap(fuzzyData.getSecond()));
builder.addFuzzyKeysData(bbpBuilder);
}
return builder.build().toByteArray();

View File

@ -21,12 +21,11 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
@ -94,7 +93,7 @@ public class InclusiveStopFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.InclusiveStopFilter.Builder builder =
FilterProtos.InclusiveStopFilter.newBuilder();
if (this.stopRowKey != null) builder.setStopRowKey(HBaseZeroCopyByteString.wrap(this.stopRowKey));
if (this.stopRowKey != null) builder.setStopRowKey(ByteStringer.wrap(this.stopRowKey));
return builder.build().toByteArray();
}

View File

@ -17,22 +17,20 @@
*/
package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.TreeSet;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.TreeSet;
/**
* This filter is used for selecting only those keys with columns that matches
@ -114,7 +112,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
FilterProtos.MultipleColumnPrefixFilter.Builder builder =
FilterProtos.MultipleColumnPrefixFilter.newBuilder();
for (byte [] element : sortedPrefixes) {
if (element != null) builder.addSortedPrefixes(HBaseZeroCopyByteString.wrap(element));
if (element != null) builder.addSortedPrefixes(ByteStringer.wrap(element));
}
return builder.build().toByteArray();
}

View File

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

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -40,7 +41,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -308,10 +308,10 @@ public class SingleColumnValueFilter extends FilterBase {
FilterProtos.SingleColumnValueFilter.Builder builder =
FilterProtos.SingleColumnValueFilter.newBuilder();
if (this.columnFamily != null) {
builder.setColumnFamily(HBaseZeroCopyByteString.wrap(this.columnFamily));
builder.setColumnFamily(ByteStringer.wrap(this.columnFamily));
}
if (this.columnQualifier != null) {
builder.setColumnQualifier(HBaseZeroCopyByteString.wrap(this.columnQualifier));
builder.setColumnQualifier(ByteStringer.wrap(this.columnQualifier));
}
HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
builder.setCompareOp(compareOp);

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -61,7 +61,7 @@ public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
final ClientProtos.CoprocessorServiceCall call =
ClientProtos.CoprocessorServiceCall.newBuilder()
.setRow(HBaseZeroCopyByteString.wrap(HConstants.EMPTY_BYTE_ARRAY))
.setRow(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY))
.setServiceName(method.getService().getFullName())
.setMethodName(method.getName())
.setRequest(request.toByteString()).build();

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -81,7 +81,7 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
final ClientProtos.CoprocessorServiceCall call =
ClientProtos.CoprocessorServiceCall.newBuilder()
.setRow(HBaseZeroCopyByteString.wrap(row))
.setRow(ByteStringer.wrap(row))
.setServiceName(method.getService().getFullName())
.setMethodName(method.getName())
.setRequest(request.toByteString()).build();

View File

@ -123,6 +123,7 @@ import org.apache.hadoop.hbase.security.access.UserPermission;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
import org.apache.hadoop.hbase.util.ExceptionUtil;
@ -859,17 +860,17 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
scanBuilder.addAttribute(attributeBuilder.build());
}
}
byte[] startRow = scan.getStartRow();
if (startRow != null && startRow.length > 0) {
scanBuilder.setStartRow(HBaseZeroCopyByteString.wrap(startRow));
scanBuilder.setStartRow(ByteStringer.wrap(startRow));
}
byte[] stopRow = scan.getStopRow();
if (stopRow != null && stopRow.length > 0) {
scanBuilder.setStopRow(HBaseZeroCopyByteString.wrap(stopRow));
scanBuilder.setStopRow(ByteStringer.wrap(stopRow));
}
if (scan.hasFilter()) {
scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
@ -878,12 +879,12 @@ public final class ProtobufUtil {
Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>>
family: scan.getFamilyMap().entrySet()) {
columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
NavigableSet<byte []> qualifiers = family.getValue();
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte [] qualifier: qualifiers) {
columnBuilder.addQualifier(HBaseZeroCopyByteString.wrap(qualifier));
columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
}
}
scanBuilder.addColumn(columnBuilder.build());
@ -997,7 +998,7 @@ public final class ProtobufUtil {
final Get get) throws IOException {
ClientProtos.Get.Builder builder =
ClientProtos.Get.newBuilder();
builder.setRow(HBaseZeroCopyByteString.wrap(get.getRow()));
builder.setRow(ByteStringer.wrap(get.getRow()));
builder.setCacheBlocks(get.getCacheBlocks());
builder.setMaxVersions(get.getMaxVersions());
if (get.getFilter() != null) {
@ -1016,7 +1017,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@ -1025,11 +1026,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(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte[] qualifier: qualifiers) {
columnBuilder.addQualifier(HBaseZeroCopyByteString.wrap(qualifier));
columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
}
}
builder.addColumn(columnBuilder.build());
@ -1062,7 +1063,7 @@ public final class ProtobufUtil {
*/
public static MutationProto toMutation(
final Increment increment, final MutationProto.Builder builder, long nonce) {
builder.setRow(HBaseZeroCopyByteString.wrap(increment.getRow()));
builder.setRow(ByteStringer.wrap(increment.getRow()));
builder.setMutateType(MutationType.INCREMENT);
builder.setDurability(toDurability(increment.getDurability()));
if (nonce != HConstants.NO_NONCE) {
@ -1079,18 +1080,18 @@ 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(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.setFamily(ByteStringer.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(HBaseZeroCopyByteString.wrap(
valueBuilder.setQualifier(ByteStringer.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
valueBuilder.setValue(HBaseZeroCopyByteString.wrap(
valueBuilder.setValue(ByteStringer.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
if (kv.getTagsLength() > 0) {
valueBuilder.setTags(HBaseZeroCopyByteString.wrap(kv.getTagsArray(),
valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(),
kv.getTagsOffset(), kv.getTagsLength()));
}
columnBuilder.addQualifierValue(valueBuilder.build());
@ -1103,7 +1104,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@ -1144,16 +1145,16 @@ public final class ProtobufUtil {
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
columnBuilder.clear();
columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family.getKey()));
columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
for (Cell cell: family.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
valueBuilder.setQualifier(HBaseZeroCopyByteString.wrap(
valueBuilder.setQualifier(ByteStringer.wrap(
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
valueBuilder.setValue(HBaseZeroCopyByteString.wrap(
valueBuilder.setValue(ByteStringer.wrap(
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
valueBuilder.setTimestamp(kv.getTimestamp());
if(cell.getTagsLength() > 0) {
valueBuilder.setTags(HBaseZeroCopyByteString.wrap(kv.getTagsArray(), kv.getTagsOffset(),
valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(), kv.getTagsOffset(),
kv.getTagsLength()));
}
if (type == MutationType.DELETE) {
@ -1214,7 +1215,7 @@ public final class ProtobufUtil {
*/
private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
final Mutation mutation, MutationProto.Builder builder) {
builder.setRow(HBaseZeroCopyByteString.wrap(mutation.getRow()));
builder.setRow(ByteStringer.wrap(mutation.getRow()));
builder.setMutateType(type);
builder.setDurability(toDurability(mutation.getDurability()));
builder.setTimestamp(mutation.getTimeStamp());
@ -1223,7 +1224,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(HBaseZeroCopyByteString.wrap(attribute.getValue()));
attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@ -1369,7 +1370,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(HBaseZeroCopyByteString.wrap(comparator.toByteArray()));
builder.setSerializedComparator(ByteStringer.wrap(comparator.toByteArray()));
return builder.build();
}
@ -1431,7 +1432,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(HBaseZeroCopyByteString.wrap(filter.toByteArray()));
builder.setSerializedFilter(ByteStringer.wrap(filter.toByteArray()));
return builder.build();
}
@ -1901,10 +1902,10 @@ public final class ProtobufUtil {
AccessControlProtos.TablePermission.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
if (tablePerm.hasFamily()) {
builder.setFamily(HBaseZeroCopyByteString.wrap(tablePerm.getFamily()));
builder.setFamily(ByteStringer.wrap(tablePerm.getFamily()));
}
if (tablePerm.hasQualifier()) {
builder.setQualifier(HBaseZeroCopyByteString.wrap(tablePerm.getQualifier()));
builder.setQualifier(ByteStringer.wrap(tablePerm.getQualifier()));
}
Permission.Action actions[] = perm.getActions();
if (actions != null) {
@ -2000,7 +2001,7 @@ public final class ProtobufUtil {
*/
public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
return AccessControlProtos.UserPermission.newBuilder()
.setUser(HBaseZeroCopyByteString.wrap(perm.getUser()))
.setUser(ByteStringer.wrap(perm.getUser()))
.setPermission(toPermission(perm))
.build();
}
@ -2256,7 +2257,7 @@ public final class ProtobufUtil {
AccessControlProtos.GetUserPermissionsRequest.Builder builder =
AccessControlProtos.GetUserPermissionsRequest.newBuilder();
if (namespace != null) {
builder.setNamespaceName(HBaseZeroCopyByteString.wrap(namespace));
builder.setNamespaceName(ByteStringer.wrap(namespace));
}
builder.setType(AccessControlProtos.Permission.Type.Namespace);
AccessControlProtos.GetUserPermissionsRequest request = builder.build();
@ -2300,8 +2301,8 @@ public final class ProtobufUtil {
*/
public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
builder.setIdentifier(HBaseZeroCopyByteString.wrap(token.getIdentifier()));
builder.setPassword(HBaseZeroCopyByteString.wrap(token.getPassword()));
builder.setIdentifier(ByteStringer.wrap(token.getIdentifier()));
builder.setPassword(ByteStringer.wrap(token.getPassword()));
if (token.getService() != null) {
builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
}
@ -2398,15 +2399,15 @@ public final class ProtobufUtil {
// Doing this is going to kill us if we do it for all data passed.
// St.Ack 20121205
CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
kvbuilder.setRow(HBaseZeroCopyByteString.wrap(kv.getRowArray(), kv.getRowOffset(),
kvbuilder.setRow(ByteStringer.wrap(kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength()));
kvbuilder.setFamily(HBaseZeroCopyByteString.wrap(kv.getFamilyArray(),
kvbuilder.setFamily(ByteStringer.wrap(kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength()));
kvbuilder.setQualifier(HBaseZeroCopyByteString.wrap(kv.getQualifierArray(),
kvbuilder.setQualifier(ByteStringer.wrap(kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength()));
kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
kvbuilder.setTimestamp(kv.getTimestamp());
kvbuilder.setValue(HBaseZeroCopyByteString.wrap(kv.getValueArray(), kv.getValueOffset(),
kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(),
kv.getValueLength()));
return kvbuilder.build();
}
@ -2485,9 +2486,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(HBaseZeroCopyByteString.wrap(info.getTableName()))
.setEncodedRegionName(HBaseZeroCopyByteString.wrap(info.getEncodedNameAsBytes()))
.setFamilyName(HBaseZeroCopyByteString.wrap(family))
.setTableName(ByteStringer.wrap(info.getTableName()))
.setEncodedRegionName(ByteStringer.wrap(info.getEncodedNameAsBytes()))
.setFamilyName(ByteStringer.wrap(family))
.setStoreHomeDir(storeDir.getName()); //make relative
for (Path inputPath : inputPaths) {
builder.addCompactionInput(inputPath.getName()); //relative path
@ -2565,8 +2566,8 @@ public final class ProtobufUtil {
public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
return HBaseProtos.TableName.newBuilder()
.setNamespace(HBaseZeroCopyByteString.wrap(tableName.getNamespace()))
.setQualifier(HBaseZeroCopyByteString.wrap(tableName.getQualifier())).build();
.setNamespace(ByteStringer.wrap(tableName.getNamespace()))
.setQualifier(ByteStringer.wrap(tableName.getQualifier())).build();
}
public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.protobuf;
import java.io.IOException;
import java.util.List;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.configuration.Configuration;
import org.apache.hadoop.classification.InterfaceAudience;
@ -138,10 +138,10 @@ public final class RequestConverter {
builder.setRegion(region);
Column.Builder columnBuilder = Column.newBuilder();
columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
columnBuilder.setFamily(ByteStringer.wrap(family));
ClientProtos.Get.Builder getBuilder =
ClientProtos.Get.newBuilder();
getBuilder.setRow(HBaseZeroCopyByteString.wrap(row));
getBuilder.setRow(ByteStringer.wrap(row));
getBuilder.addColumn(columnBuilder.build());
getBuilder.setClosestRowBefore(true);
builder.setGet(getBuilder.build());
@ -186,14 +186,14 @@ public final class RequestConverter {
builder.setRegion(region);
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(HBaseZeroCopyByteString.wrap(row));
mutateBuilder.setRow(ByteStringer.wrap(row));
mutateBuilder.setMutateType(MutationType.INCREMENT);
mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
columnBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
columnBuilder.setFamily(ByteStringer.wrap(family));
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
valueBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(amount)));
valueBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
valueBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(amount)));
valueBuilder.setQualifier(ByteStringer.wrap(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build());
if (nonce != HConstants.NO_NONCE) {
@ -496,7 +496,7 @@ public final class RequestConverter {
builder.setRegion(region);
FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder();
for (Pair<byte[], String> familyPath: familyPaths) {
familyPathBuilder.setFamily(HBaseZeroCopyByteString.wrap(familyPath.getFirst()));
familyPathBuilder.setFamily(ByteStringer.wrap(familyPath.getFirst()));
familyPathBuilder.setPath(familyPath.getSecond());
builder.addFamilyPath(familyPathBuilder.build());
}
@ -541,7 +541,7 @@ public final class RequestConverter {
RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
regionActionBuilder.addAction(actionBuilder.setServiceCall(
ClientProtos.CoprocessorServiceCall.newBuilder()
.setRow(HBaseZeroCopyByteString.wrap(exec.getRow()))
.setRow(ByteStringer.wrap(exec.getRow()))
.setServiceName(exec.getMethod().getService().getFullName())
.setMethodName(exec.getMethod().getName())
.setRequest(exec.getRequest().toByteString())));
@ -672,7 +672,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.addFamily(HBaseZeroCopyByteString.wrap(family));
builder.addFamily(ByteStringer.wrap(family));
return builder.build();
}
@ -835,7 +835,7 @@ public final class RequestConverter {
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
if (splitPoint != null) {
builder.setSplitPoint(HBaseZeroCopyByteString.wrap(splitPoint));
builder.setSplitPoint(ByteStringer.wrap(splitPoint));
}
return builder.build();
}
@ -875,7 +875,7 @@ public final class RequestConverter {
builder.setRegion(region);
builder.setMajor(major);
if (family != null) {
builder.setFamily(HBaseZeroCopyByteString.wrap(family));
builder.setFamily(ByteStringer.wrap(family));
}
return builder.build();
}
@ -934,7 +934,7 @@ public final class RequestConverter {
public static RegionSpecifier buildRegionSpecifier(
final RegionSpecifierType type, final byte[] value) {
RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder();
regionBuilder.setValue(HBaseZeroCopyByteString.wrap(value));
regionBuilder.setValue(ByteStringer.wrap(value));
regionBuilder.setType(type);
return regionBuilder.build();
}
@ -955,9 +955,9 @@ public final class RequestConverter {
final ByteArrayComparable comparator,
final CompareType compareType) throws IOException {
Condition.Builder builder = Condition.newBuilder();
builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.setFamily(HBaseZeroCopyByteString.wrap(family));
builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
builder.setRow(ByteStringer.wrap(row));
builder.setFamily(ByteStringer.wrap(family));
builder.setQualifier(ByteStringer.wrap(qualifier));
builder.setComparator(ProtobufUtil.toComparator(comparator));
builder.setCompareType(compareType);
return builder.build();
@ -989,7 +989,7 @@ public final class RequestConverter {
final TableName tableName, final byte [] columnName) {
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnName(HBaseZeroCopyByteString.wrap(columnName));
builder.setColumnName(ByteStringer.wrap(columnName));
return builder.build();
}
@ -1144,7 +1144,7 @@ public final class RequestConverter {
builder.setTableSchema(hTableDesc.convert());
if (splitKeys != null) {
for (byte [] splitKey : splitKeys) {
builder.addSplitKeys(HBaseZeroCopyByteString.wrap(splitKey));
builder.addSplitKeys(ByteStringer.wrap(splitKey));
}
}
return builder.build();
@ -1297,7 +1297,7 @@ public final class RequestConverter {
public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest(
byte[] regionName) {
return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName(
HBaseZeroCopyByteString.wrap(regionName)).build();
ByteStringer.wrap(regionName)).build();
}
/**
@ -1352,10 +1352,10 @@ public final class RequestConverter {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (family != null) {
permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
permissionBuilder.setFamily(ByteStringer.wrap(family));
}
if (qualifier != null) {
permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
permissionBuilder.setQualifier(ByteStringer.wrap(qualifier));
}
ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder);
@ -1448,10 +1448,10 @@ public final class RequestConverter {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (family != null) {
permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
permissionBuilder.setFamily(ByteStringer.wrap(family));
}
if (qualifier != null) {
permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
permissionBuilder.setQualifier(ByteStringer.wrap(qualifier));
}
ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder);

View File

@ -26,7 +26,7 @@ import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -83,15 +83,15 @@ public class EncryptionUtil {
if (cipher.getIvLength() > 0) {
iv = new byte[cipher.getIvLength()];
RNG.nextBytes(iv);
builder.setIv(HBaseZeroCopyByteString.wrap(iv));
builder.setIv(ByteStringer.wrap(iv));
}
byte[] keyBytes = key.getEncoded();
builder.setLength(keyBytes.length);
builder.setHash(HBaseZeroCopyByteString.wrap(Encryption.hash128(keyBytes)));
builder.setHash(ByteStringer.wrap(Encryption.hash128(keyBytes)));
ByteArrayOutputStream out = new ByteArrayOutputStream();
Encryption.encryptWithSubjectKey(out, new ByteArrayInputStream(keyBytes), subject,
conf, cipher, iv);
builder.setData(HBaseZeroCopyByteString.wrap(out.toByteArray()));
builder.setData(ByteStringer.wrap(out.toByteArray()));
// Build and return the protobuf message
out.reset();
builder.build().writeDelimitedTo(out);

View File

@ -23,7 +23,7 @@ import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -93,10 +93,10 @@ public class AccessControlClient {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (family != null) {
permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
permissionBuilder.setFamily(ByteStringer.wrap(family));
}
if (qual != null) {
permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qual));
permissionBuilder.setQualifier(ByteStringer.wrap(qual));
}
ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission(
permissionBuilder);
@ -157,10 +157,10 @@ public class AccessControlClient {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (family != null) {
permissionBuilder.setFamily(HBaseZeroCopyByteString.wrap(family));
permissionBuilder.setFamily(ByteStringer.wrap(family));
}
if (qualifier != null) {
permissionBuilder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
permissionBuilder.setQualifier(ByteStringer.wrap(qualifier));
}
ret.setType(AccessControlProtos.Permission.Type.Table).setTablePermission(
permissionBuilder);

View File

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

View File

@ -35,7 +35,7 @@ import java.util.Properties;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -1968,7 +1968,7 @@ public class ZKUtil {
for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
byte[] columnFamilyName = e.getKey();
Long curSeqId = e.getValue();
storeSequenceIdBuilder.setFamilyName(HBaseZeroCopyByteString.wrap(columnFamilyName));
storeSequenceIdBuilder.setFamilyName(ByteStringer.wrap(columnFamilyName));
storeSequenceIdBuilder.setSequenceId(curSeqId);
regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
storeSequenceIdBuilder.clear();

View File

@ -33,7 +33,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.lang.NotImplementedException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -512,7 +512,7 @@ public class TestClientNoCluster extends Configured implements Tool {
if (max <= 0) break;
if (++count > max) break;
HRegionInfo hri = e.getValue().getFirst();
ByteString row = HBaseZeroCopyByteString.wrap(hri.getRegionName());
ByteString row = ByteStringer.wrap(hri.getRegionName());
resultBuilder.clear();
resultBuilder.addCell(getRegionInfo(row, hri));
resultBuilder.addCell(getServer(row, e.getValue().getSecond()));
@ -567,11 +567,11 @@ public class TestClientNoCluster extends Configured implements Tool {
}
private final static ByteString CATALOG_FAMILY_BYTESTRING =
HBaseZeroCopyByteString.wrap(HConstants.CATALOG_FAMILY);
ByteStringer.wrap(HConstants.CATALOG_FAMILY);
private final static ByteString REGIONINFO_QUALIFIER_BYTESTRING =
HBaseZeroCopyByteString.wrap(HConstants.REGIONINFO_QUALIFIER);
ByteStringer.wrap(HConstants.REGIONINFO_QUALIFIER);
private final static ByteString SERVER_QUALIFIER_BYTESTRING =
HBaseZeroCopyByteString.wrap(HConstants.SERVER_QUALIFIER);
ByteStringer.wrap(HConstants.SERVER_QUALIFIER);
static CellProtos.Cell.Builder getBaseCellBuilder(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = CellProtos.Cell.newBuilder();
@ -584,7 +584,7 @@ public class TestClientNoCluster extends Configured implements Tool {
static CellProtos.Cell getRegionInfo(final ByteString row, final HRegionInfo hri) {
CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row);
cellBuilder.setQualifier(REGIONINFO_QUALIFIER_BYTESTRING);
cellBuilder.setValue(HBaseZeroCopyByteString.wrap(hri.toByteArray()));
cellBuilder.setValue(ByteStringer.wrap(hri.toByteArray()));
return cellBuilder.build();
}
@ -597,9 +597,9 @@ public class TestClientNoCluster extends Configured implements Tool {
static CellProtos.Cell getStartCode(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row);
cellBuilder.setQualifier(HBaseZeroCopyByteString.wrap(HConstants.STARTCODE_QUALIFIER));
cellBuilder.setQualifier(ByteStringer.wrap(HConstants.STARTCODE_QUALIFIER));
// TODO:
cellBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(META_SERVERNAME.getStartcode())));
cellBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(META_SERVERNAME.getStartcode())));
return cellBuilder.build();
}

View File

@ -108,6 +108,10 @@
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</dependency>
</dependencies>
<profiles>

View File

@ -64,5 +64,4 @@ public final class HBaseZeroCopyByteString extends LiteralByteString {
throw new UnsupportedOperationException("Need a LiteralByteString, got a "
+ buf.getClass().getName());
}
}
}

View File

@ -0,0 +1,67 @@
/**
* 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 org.apache.hadoop.hbase.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
/**
* Hack to workaround HBASE-1304 issue that keeps bubbling up when a mapreduce context.
*/
public class ByteStringer {
private static final Log LOG = LogFactory.getLog(ByteStringer.class);
/**
* Flag set at class loading time.
*/
private static boolean USE_ZEROCOPYBYTESTRING = true;
// Can I classload HBaseZeroCopyByteString without IllegalAccessError?
// If we can, use it passing ByteStrings to pb else use native ByteString though more costly
// because it makes a copy of the passed in array.
static {
try {
HBaseZeroCopyByteString.wrap(new byte [0]);
} catch (IllegalAccessError iae) {
USE_ZEROCOPYBYTESTRING = false;
LOG.debug("Failed to classload HBaseZeroCopyByteString: " + iae.toString());
}
}
private ByteStringer() {
super();
}
/**
* Wraps a byte array in a {@link ByteString} without copying it.
*/
public static ByteString wrap(final byte[] array) {
return USE_ZEROCOPYBYTESTRING? HBaseZeroCopyByteString.wrap(array): ByteString.copyFrom(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 USE_ZEROCOPYBYTESTRING? HBaseZeroCopyByteString.wrap(array, offset, length):
ByteString.copyFrom(array, offset, length);
}
}

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
@ -44,15 +44,15 @@ public class MessageCodec implements Codec {
CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
// This copies bytes from Cell to ByteString. I don't see anyway around the copy.
// ByteString is final.
builder.setRow(HBaseZeroCopyByteString.wrap(cell.getRowArray(), cell.getRowOffset(),
builder.setRow(ByteStringer.wrap(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength()));
builder.setFamily(HBaseZeroCopyByteString.wrap(cell.getFamilyArray(), cell.getFamilyOffset(),
builder.setFamily(ByteStringer.wrap(cell.getFamilyArray(), cell.getFamilyOffset(),
cell.getFamilyLength()));
builder.setQualifier(HBaseZeroCopyByteString.wrap(cell.getQualifierArray(),
builder.setQualifier(ByteStringer.wrap(cell.getQualifierArray(),
cell.getQualifierOffset(), cell.getQualifierLength()));
builder.setTimestamp(cell.getTimestamp());
builder.setCellType(CellProtos.CellType.valueOf(cell.getTypeByte()));
builder.setValue(HBaseZeroCopyByteString.wrap(cell.getValueArray(), cell.getValueOffset(),
builder.setValue(ByteStringer.wrap(cell.getValueArray(), cell.getValueOffset(),
cell.getValueLength()));
CellProtos.Cell pbcell = builder.build();
pbcell.writeDelimitedTo(this.out);

View File

@ -24,6 +24,7 @@ import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -33,8 +34,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.HBaseZeroCopyByteString;
/**
* 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
@ -194,7 +193,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(HBaseZeroCopyByteString.wrap(getSplitKey()));
builder.setSplitkey(ByteStringer.wrap(getSplitKey()));
return builder.build();
}

View File

@ -26,7 +26,7 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hbase.KeyValue;
@ -205,7 +205,7 @@ public class FixedFileTrailer {
.setComparatorClassName(comparatorClassName)
.setCompressionCodec(compressionCodec.ordinal());
if (encryptionKey != null) {
builder.setEncryptionKey(HBaseZeroCopyByteString.wrap(encryptionKey));
builder.setEncryptionKey(ByteStringer.wrap(encryptionKey));
}
// We need this extra copy unfortunately to determine the final size of the
// delimited output, see use of baos.size() below.

View File

@ -40,7 +40,7 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -759,8 +759,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(HBaseZeroCopyByteString.wrap(e.getKey()));
bbpBuilder.setSecond(HBaseZeroCopyByteString.wrap(e.getValue()));
bbpBuilder.setFirst(ByteStringer.wrap(e.getKey()));
bbpBuilder.setSecond(ByteStringer.wrap(e.getValue()));
builder.addMapEntry(bbpBuilder.build());
}
out.write(ProtobufUtil.PB_MAGIC);

View File

@ -1238,7 +1238,6 @@ public class HFileBlock implements Cacheable {
} else {
// Positional read. Better for random reads; or when the streamLock is already locked.
int extraSize = peekIntoNextBlock ? hdrSize : 0;
int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
if (ret < size) {
throw new IOException("Positional read of " + size + " bytes " +
@ -1252,8 +1251,7 @@ public class HFileBlock implements Cacheable {
}
assert peekIntoNextBlock;
return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) +
hdrSize;
return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) + hdrSize;
}
}

View File

@ -28,7 +28,7 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.UUID;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
@ -91,8 +91,8 @@ public class ReplicationProtbufUtil {
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
HLogKey key = entry.getKey();
keyBuilder.setEncodedRegionName(
HBaseZeroCopyByteString.wrap(key.getEncodedRegionName()));
keyBuilder.setTableName(HBaseZeroCopyByteString.wrap(key.getTablename().getName()));
ByteStringer.wrap(key.getEncodedRegionName()));
keyBuilder.setTableName(ByteStringer.wrap(key.getTablename().getName()));
keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
keyBuilder.setWriteTime(key.getWriteTime());
if (key.getNonce() != HConstants.NO_NONCE) {
@ -113,7 +113,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(HBaseZeroCopyByteString.wrap(scope.getKey()));
scopeBuilder.setFamily(ByteStringer.wrap(scope.getKey()));
WALProtos.ScopeType scopeType =
WALProtos.ScopeType.valueOf(scope.getValue().intValue());
scopeBuilder.setScopeType(scopeType);

View File

@ -121,6 +121,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.ConfigUtil;
@ -149,7 +150,6 @@ import org.apache.zookeeper.data.Stat;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.ServiceException;
/**
@ -1271,7 +1271,7 @@ public class HRegionServer extends HasThread implements
regionSpecifier = RegionSpecifier.newBuilder();
}
regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
regionSpecifier.setValue(HBaseZeroCopyByteString.wrap(name));
regionSpecifier.setValue(ByteStringer.wrap(name));
regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
.setStores(stores)
.setStorefiles(storefiles)

View File

@ -35,8 +35,7 @@ import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -543,8 +542,8 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceNumber {
throws IOException {
WALKey.Builder builder = WALKey.newBuilder();
if (compressionContext == null) {
builder.setEncodedRegionName(HBaseZeroCopyByteString.wrap(this.encodedRegionName));
builder.setTableName(HBaseZeroCopyByteString.wrap(this.tablename.getName()));
builder.setEncodedRegionName(ByteStringer.wrap(this.encodedRegionName));
builder.setTableName(ByteStringer.wrap(this.tablename.getName()));
} else {
builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
compressionContext.regionDict));
@ -570,8 +569,7 @@ public class HLogKey implements WritableComparable<HLogKey>, SequenceNumber {
}
if (scopes != null) {
for (Map.Entry<byte[], Integer> e : scopes.entrySet()) {
ByteString family = (compressionContext == null) ?
HBaseZeroCopyByteString.wrap(e.getKey())
ByteString family = (compressionContext == null) ? ByteStringer.wrap(e.getKey())
: compressor.compress(e.getKey(), compressionContext.familyDict);
builder.addScopes(FamilyScope.newBuilder()
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));

View File

@ -24,7 +24,7 @@ import java.security.SecureRandom;
import javax.crypto.spec.SecretKeySpec;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
@ -57,7 +57,7 @@ public class SecureProtobufLogWriter extends ProtobufLogWriter {
byte[] keyBytes = new byte[cipher.getKeyLength()];
rng.nextBytes(keyBytes);
Key key = new SecretKeySpec(keyBytes, cipher.getName());
builder.setEncryptionKey(HBaseZeroCopyByteString.wrap(EncryptionUtil.wrapKey(conf,
builder.setEncryptionKey(ByteStringer.wrap(EncryptionUtil.wrapKey(conf,
conf.get(HConstants.CRYPTO_WAL_KEY_NAME_CONF_KEY,
conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
User.getCurrent().getShortName())),

View File

@ -28,7 +28,7 @@ import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.bind.annotation.XmlValue;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
@ -184,8 +184,8 @@ public class CellModel implements ProtobufMessageHandler, Serializable {
@Override
public byte[] createProtobufOutput() {
Cell.Builder builder = Cell.newBuilder();
builder.setColumn(HBaseZeroCopyByteString.wrap(getColumn()));
builder.setData(HBaseZeroCopyByteString.wrap(getValue()));
builder.setColumn(ByteStringer.wrap(getColumn()));
builder.setData(ByteStringer.wrap(getValue()));
if (hasUserTimestamp()) {
builder.setTimestamp(getTimestamp());
}

View File

@ -29,7 +29,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
@ -114,11 +114,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(HBaseZeroCopyByteString.wrap(row.getKey()));
rowBuilder.setKey(ByteStringer.wrap(row.getKey()));
for (CellModel cell: row.getCells()) {
Cell.Builder cellBuilder = Cell.newBuilder();
cellBuilder.setColumn(HBaseZeroCopyByteString.wrap(cell.getColumn()));
cellBuilder.setData(HBaseZeroCopyByteString.wrap(cell.getValue()));
cellBuilder.setColumn(ByteStringer.wrap(cell.getColumn()));
cellBuilder.setData(ByteStringer.wrap(cell.getValue()));
if (cell.hasUserTimestamp()) {
cellBuilder.setTimestamp(cell.getTimestamp());
}

View File

@ -75,7 +75,7 @@ import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.sun.jersey.api.json.JSONConfiguration;
import com.sun.jersey.api.json.JSONJAXBContext;
import com.sun.jersey.api.json.JSONMarshaller;
@ -781,13 +781,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(HBaseZeroCopyByteString.wrap(startRow));
builder.setStartRow(ByteStringer.wrap(startRow));
}
if (!Bytes.equals(endRow, HConstants.EMPTY_START_ROW)) {
builder.setEndRow(HBaseZeroCopyByteString.wrap(endRow));
builder.setEndRow(ByteStringer.wrap(endRow));
}
for (byte[] column: columns) {
builder.addColumns(HBaseZeroCopyByteString.wrap(column));
builder.addColumns(ByteStringer.wrap(column));
}
if (startTime != 0) {
builder.setStartTime(startTime);

View File

@ -29,7 +29,7 @@ import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlElementWrapper;
import javax.xml.bind.annotation.XmlRootElement;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
@ -721,7 +721,7 @@ public class StorageClusterStatusModel
for (Node.Region region: node.regions) {
StorageClusterStatus.Region.Builder regionBuilder =
StorageClusterStatus.Region.newBuilder();
regionBuilder.setName(HBaseZeroCopyByteString.wrap(region.name));
regionBuilder.setName(ByteStringer.wrap(region.name));
regionBuilder.setStores(region.stores);
regionBuilder.setStorefiles(region.storefiles);
regionBuilder.setStorefileSizeMB(region.storefileSizeMB);

View File

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

View File

@ -122,7 +122,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
@ -1427,7 +1427,7 @@ public class VisibilityController extends BaseRegionObserver implements MasterOb
try {
List<String> labels = getUserAuthsFromLabelsTable(user);
for (String label : labels) {
response.addAuth(HBaseZeroCopyByteString.wrap(Bytes.toBytes(label)));
response.addAuth(ByteStringer.wrap(Bytes.toBytes(label)));
}
} catch (IOException e) {
ResponseConverter.setControllerException(controller, e);

View File

@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUs
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.ReflectionUtils;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -71,7 +71,7 @@ public class VisibilityUtils {
VisibilityLabelsRequest.Builder visReqBuilder = VisibilityLabelsRequest.newBuilder();
for (Entry<String, Integer> entry : existingLabels.entrySet()) {
VisibilityLabel.Builder visLabBuilder = VisibilityLabel.newBuilder();
visLabBuilder.setLabel(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
visLabBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(entry.getKey())));
visLabBuilder.setOrdinal(entry.getValue());
visReqBuilder.addVisLabel(visLabBuilder.build());
}
@ -87,7 +87,7 @@ public class VisibilityUtils {
MultiUserAuthorizations.Builder builder = MultiUserAuthorizations.newBuilder();
for (Entry<String, List<Integer>> entry : userAuths.entrySet()) {
UserAuthorizations.Builder userAuthsBuilder = UserAuthorizations.newBuilder();
userAuthsBuilder.setUser(HBaseZeroCopyByteString.wrap(Bytes.toBytes(entry.getKey())));
userAuthsBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(entry.getKey())));
for (Integer label : entry.getValue()) {
userAuthsBuilder.addAuth(label);
}

View File

@ -19,14 +19,13 @@
package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -111,9 +110,9 @@ public class TestBatchCoprocessorEndpoint {
ColumnAggregationWithNullResponseProtos.SumRequest.Builder builder =
ColumnAggregationWithNullResponseProtos.SumRequest
.newBuilder();
builder.setFamily(HBaseZeroCopyByteString.wrap(TEST_FAMILY));
builder.setFamily(ByteStringer.wrap(TEST_FAMILY));
if (TEST_QUALIFIER != null && TEST_QUALIFIER.length > 0) {
builder.setQualifier(HBaseZeroCopyByteString.wrap(TEST_QUALIFIER));
builder.setQualifier(ByteStringer.wrap(TEST_QUALIFIER));
}
Map<byte[], ColumnAggregationWithNullResponseProtos.SumResponse> results =
table.batchCoprocessorService(
@ -149,9 +148,9 @@ public class TestBatchCoprocessorEndpoint {
Throwable {
ColumnAggregationProtos.SumRequest.Builder builder = ColumnAggregationProtos.SumRequest
.newBuilder();
builder.setFamily(HBaseZeroCopyByteString.wrap(family));
builder.setFamily(ByteStringer.wrap(family));
if (qualifier != null && qualifier.length > 0) {
builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
builder.setQualifier(ByteStringer.wrap(qualifier));
}
return table.batchCoprocessorService(
ColumnAggregationProtos.ColumnAggregationService.getDescriptor().findMethodByName("sum"),
@ -238,9 +237,9 @@ public class TestBatchCoprocessorEndpoint {
ColumnAggregationWithErrorsProtos.SumRequest.Builder builder =
ColumnAggregationWithErrorsProtos.SumRequest
.newBuilder();
builder.setFamily(HBaseZeroCopyByteString.wrap(TEST_FAMILY));
builder.setFamily(ByteStringer.wrap(TEST_FAMILY));
if (TEST_QUALIFIER != null && TEST_QUALIFIER.length > 0) {
builder.setQualifier(HBaseZeroCopyByteString.wrap(TEST_QUALIFIER));
builder.setQualifier(ByteStringer.wrap(TEST_QUALIFIER));
}
boolean hasError = false;

View File

@ -30,7 +30,7 @@ import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -125,9 +125,9 @@ public class TestCoprocessorEndpoint {
new BlockingRpcCallback<ColumnAggregationProtos.SumResponse>();
ColumnAggregationProtos.SumRequest.Builder builder =
ColumnAggregationProtos.SumRequest.newBuilder();
builder.setFamily(HBaseZeroCopyByteString.wrap(family));
builder.setFamily(ByteStringer.wrap(family));
if (qualifier != null && qualifier.length > 0) {
builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier));
builder.setQualifier(ByteStringer.wrap(qualifier));
}
instance.sum(null, builder.build(), rpcCallback);
return rpcCallback.get().getSum();

View File

@ -33,7 +33,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
@ -366,7 +366,7 @@ public class TestRowProcessorEndpoint {
public IncCounterProcessorRequest getRequestData() throws IOException {
IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
builder.setCounter(counter);
builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.setRow(ByteStringer.wrap(row));
return builder.build();
}
@ -445,8 +445,8 @@ public class TestRowProcessorEndpoint {
public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
FriendsOfFriendsProcessorRequest.Builder builder =
FriendsOfFriendsProcessorRequest.newBuilder();
builder.setPerson(HBaseZeroCopyByteString.wrap(person));
builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.setPerson(ByteStringer.wrap(person));
builder.setRow(ByteStringer.wrap(row));
builder.addAllResult(result);
FriendsOfFriendsProcessorRequest f = builder.build();
return f;
@ -554,8 +554,8 @@ public class TestRowProcessorEndpoint {
@Override
public RowSwapProcessorRequest getRequestData() throws IOException {
RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
builder.setRow1(HBaseZeroCopyByteString.wrap(row1));
builder.setRow2(HBaseZeroCopyByteString.wrap(row2));
builder.setRow1(ByteStringer.wrap(row1));
builder.setRow2(ByteStringer.wrap(row2));
return builder.build();
}
@ -614,7 +614,7 @@ public class TestRowProcessorEndpoint {
@Override
public TimeoutProcessorRequest getRequestData() throws IOException {
TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
builder.setRow(HBaseZeroCopyByteString.wrap(row));
builder.setRow(ByteStringer.wrap(row));
return builder.build();
}

View File

@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Append;
@ -55,12 +55,12 @@ public class TestProtobufUtil {
NameBytesPair.Builder builder = NameBytesPair.newBuilder();
final String omg = "OMG!!!";
builder.setName("java.io.IOException");
builder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(omg)));
builder.setValue(ByteStringer.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(HBaseZeroCopyByteString.wrap(Bytes.toBytes(omg)));
builder.setValue(ByteStringer.wrap(Bytes.toBytes(omg)));
t = ProtobufUtil.toException(builder.build());
assertEquals(omg, t.getMessage());
}
@ -200,10 +200,10 @@ public class TestProtobufUtil {
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
qualifierBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(11L)));
qualifierBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(11L)));
valueBuilder.addQualifierValue(qualifierBuilder.build());
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
qualifierBuilder.setValue(HBaseZeroCopyByteString.wrap(Bytes.toBytes(22L)));
qualifierBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(22L)));
valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build());

View File

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

View File

@ -25,7 +25,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -261,9 +261,9 @@ public class TestReplicationSink {
uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
keyBuilder.setClusterId(uuidBuilder.build());
keyBuilder.setTableName(HBaseZeroCopyByteString.wrap(table));
keyBuilder.setTableName(ByteStringer.wrap(table));
keyBuilder.setWriteTime(now);
keyBuilder.setEncodedRegionName(HBaseZeroCopyByteString.wrap(HConstants.EMPTY_BYTE_ARRAY));
keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY));
keyBuilder.setLogSequenceNumber(-1);
builder.setKey(keyBuilder.build());
cells.add(kv);