You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2016/09/30 21:57:18 UTC
[29/29] hbase git commit: HBASE-16741 Amend the generate protobufs
out-of-band build step to include shade,
pulling in protobuf source and a hook for patching protobuf
HBASE-16741 Amend the generate protobufs out-of-band build step to include shade, pulling in protobuf source and a hook for patching protobuf
Removed ByteStringer from hbase-protocol-shaded. Use the protobuf-3.1.0
trick directly instead. Makes stuff cleaner. All under 'shaded' dir is
now generated.
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/32be831c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/32be831c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/32be831c
Branch: refs/heads/HBASE-16264
Commit: 32be831ce56beab404d463cd7ada54a98f9e99f8
Parents: 35cf550
Author: stack <st...@apache.org>
Authored: Fri Sep 30 14:54:58 2016 -0700
Committer: stack <st...@apache.org>
Committed: Fri Sep 30 14:54:58 2016 -0700
----------------------------------------------------------------------
.../org/apache/hadoop/hbase/HRegionInfo.java | 6 +-
.../hbase/filter/ColumnPaginationFilter.java | 4 +-
.../hadoop/hbase/filter/ColumnPrefixFilter.java | 4 +-
.../hadoop/hbase/filter/ColumnRangeFilter.java | 10 +-
.../hbase/filter/DependentColumnFilter.java | 6 +-
.../FirstKeyValueMatchingQualifiersFilter.java | 4 +-
.../hadoop/hbase/filter/FuzzyRowFilter.java | 8 +-
.../hbase/filter/InclusiveStopFilter.java | 7 +-
.../hbase/filter/MultiRowRangeFilter.java | 9 +-
.../filter/MultipleColumnPrefixFilter.java | 7 +-
.../hadoop/hbase/filter/PrefixFilter.java | 4 +-
.../hbase/filter/SingleColumnValueFilter.java | 8 +-
.../hadoop/hbase/security/EncryptionUtil.java | 8 +-
.../hbase/shaded/protobuf/ProtobufUtil.java | 106 +-
.../hbase/shaded/protobuf/RequestConverter.java | 36 +-
.../apache/hadoop/hbase/zookeeper/ZKUtil.java | 11 +-
.../hbase/client/TestClientNoCluster.java | 17 +-
.../hadoop/hbase/procedure2/ProcedureUtil.java | 8 +-
hbase-protocol-shaded/README.txt | 77 +-
hbase-protocol-shaded/pom.xml | 558 +-
.../protobuf/generated/TestProcedureProtos.java | 178 +-
.../ipc/protobuf/generated/TestProtos.java | 858 +-
.../generated/TestRpcServiceProtos.java | 226 +-
.../shaded/protobuf/generated/AdminProtos.java | 5566 +++---
.../shaded/protobuf/generated/CellProtos.java | 474 +-
.../shaded/protobuf/generated/ClientProtos.java | 6168 +++---
.../protobuf/generated/ClusterIdProtos.java | 178 +-
.../protobuf/generated/ClusterStatusProtos.java | 1620 +-
.../protobuf/generated/ComparatorProtos.java | 1378 +-
.../protobuf/generated/EncryptionProtos.java | 234 +-
.../protobuf/generated/ErrorHandlingProtos.java | 632 +-
.../shaded/protobuf/generated/FSProtos.java | 336 +-
.../shaded/protobuf/generated/FilterProtos.java | 4210 ++---
.../shaded/protobuf/generated/HBaseProtos.java | 4144 ++---
.../shaded/protobuf/generated/HFileProtos.java | 366 +-
.../protobuf/generated/LoadBalancerProtos.java | 148 +-
.../protobuf/generated/MapReduceProtos.java | 326 +-
.../generated/MasterProcedureProtos.java | 2986 +--
.../shaded/protobuf/generated/MasterProtos.java | 16494 ++++++++---------
.../protobuf/generated/ProcedureProtos.java | 1246 +-
.../shaded/protobuf/generated/QuotaProtos.java | 760 +-
.../shaded/protobuf/generated/RPCProtos.java | 1128 +-
.../generated/RegionNormalizerProtos.java | 148 +-
.../generated/RegionServerStatusProtos.java | 1804 +-
.../protobuf/generated/SnapshotProtos.java | 870 +-
.../protobuf/generated/TracingProtos.java | 152 +-
.../shaded/protobuf/generated/WALProtos.java | 2030 +-
.../protobuf/generated/ZooKeeperProtos.java | 1744 +-
.../hadoop/hbase/shaded/util/ByteStringer.java | 48 -
.../src/main/protobuf/RSGroup.proto | 34 -
.../src/main/protobuf/RSGroupAdmin.proto | 136 -
.../apache/hadoop/hbase/codec/MessageCodec.java | 11 +-
.../org/apache/hadoop/hbase/io/Reference.java | 4 +-
.../hadoop/hbase/io/hfile/FixedFileTrailer.java | 8 +-
.../org/apache/hadoop/hbase/io/hfile/HFile.java | 12 +-
.../hadoop/hbase/master/MasterRpcServices.java | 4 +-
.../procedure/DeleteColumnFamilyProcedure.java | 4 +-
.../hbase/protobuf/ReplicationProtbufUtil.java | 10 +-
.../hadoop/hbase/regionserver/HRegion.java | 35 +-
.../hbase/regionserver/HRegionServer.java | 8 +-
.../wal/AbstractProtobufLogWriter.java | 8 +-
.../hbase/snapshot/SnapshotManifestV1.java | 12 +-
.../hbase/snapshot/SnapshotManifestV2.java | 5 +-
.../hbase/regionserver/TestHRegionInfo.java | 10 +-
.../regionserver/TestHRegionReplayEvents.java | 41 +-
.../hbase/regionserver/TestPriorityRpc.java | 7 +-
.../regionserver/TestReplicationSink.java | 9 +-
.../TestReplicationSourceManager.java | 4 +-
.../hbase/snapshot/TestSnapshotManifest.java | 4 +-
pom.xml | 3 +-
70 files changed, 28784 insertions(+), 28935 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 09e53ce..da0d941 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -33,11 +33,11 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.util.ByteArrayHashKey;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HashKey;
import org.apache.hadoop.hbase.util.JenkinsHash;
@@ -873,10 +873,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) {
- builder.setStartKey(ByteStringer.wrap(info.getStartKey()));
+ builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey()));
}
if (info.getEndKey() != null) {
- builder.setEndKey(ByteStringer.wrap(info.getEndKey()));
+ builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey()));
}
builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit());
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
index d2255d1..7d4571e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
@@ -28,11 +28,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
@@ -172,7 +172,7 @@ public class ColumnPaginationFilter extends FilterBase {
builder.setOffset(this.offset);
}
if (this.columnOffset != null) {
- builder.setColumnOffset(ByteStringer.wrap(this.columnOffset));
+ builder.setColumnOffset(UnsafeByteOperations.unsafeWrap(this.columnOffset));
}
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
index 46fb82d..806841f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
@@ -30,11 +30,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* This filter is used for selecting only those keys with columns that matches
@@ -112,7 +112,7 @@ public class ColumnPrefixFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.ColumnPrefixFilter.Builder builder =
FilterProtos.ColumnPrefixFilter.newBuilder();
- if (this.prefix != null) builder.setPrefix(ByteStringer.wrap(this.prefix));
+ if (this.prefix != null) builder.setPrefix(UnsafeByteOperations.unsafeWrap(this.prefix));
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
index 78d7fb5..99f9926 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
@@ -30,12 +30,12 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used for selecting only those keys with columns that are
@@ -175,9 +175,11 @@ public class ColumnRangeFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.ColumnRangeFilter.Builder builder =
FilterProtos.ColumnRangeFilter.newBuilder();
- if (this.minColumn != null) builder.setMinColumn(ByteStringer.wrap(this.minColumn));
+ if (this.minColumn != null) builder.setMinColumn(
+ UnsafeByteOperations.unsafeWrap(this.minColumn));
builder.setMinColumnInclusive(this.minColumnInclusive);
- if (this.maxColumn != null) builder.setMaxColumn(ByteStringer.wrap(this.maxColumn));
+ if (this.maxColumn != null) builder.setMaxColumn(
+ UnsafeByteOperations.unsafeWrap(this.maxColumn));
builder.setMaxColumnInclusive(this.maxColumnInclusive);
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
index f273e63..287a090 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
@@ -32,12 +32,12 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* A filter for adding inter-column timestamp matching
@@ -222,10 +222,10 @@ public class DependentColumnFilter extends CompareFilter {
FilterProtos.DependentColumnFilter.newBuilder();
builder.setCompareFilter(super.convert());
if (this.columnFamily != null) {
- builder.setColumnFamily(ByteStringer.wrap(this.columnFamily));
+ builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily));
}
if (this.columnQualifier != null) {
- builder.setColumnQualifier(ByteStringer.wrap(this.columnQualifier));
+ builder.setColumnQualifier(UnsafeByteOperations.unsafeWrap(this.columnQualifier));
}
builder.setDropDependentColumn(this.dropDependentColumn);
return builder.build().toByteArray();
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
index 98513f5..82d6c57 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* The filter looks for the given columns in KeyValue. Once there is a match for
@@ -88,7 +88,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
for (byte[] qualifier : qualifiers) {
- if (qualifier != null) builder.addQualifiers(ByteStringer.wrap(qualifier));
+ if (qualifier != null) builder.addQualifiers(UnsafeByteOperations.unsafeWrap(qualifier));
}
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index e8a75db..5fc12b9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -29,16 +29,16 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.UnsafeAccess;
import org.apache.hadoop.hbase.util.UnsafeAvailChecker;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
/**
* This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key.
@@ -257,8 +257,8 @@ public class FuzzyRowFilter extends FilterBase {
FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder();
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
- bbpBuilder.setFirst(ByteStringer.wrap(fuzzyData.getFirst()));
- bbpBuilder.setSecond(ByteStringer.wrap(fuzzyData.getSecond()));
+ bbpBuilder.setFirst(UnsafeByteOperations.unsafeWrap(fuzzyData.getFirst()));
+ bbpBuilder.setSecond(UnsafeByteOperations.unsafeWrap(fuzzyData.getSecond()));
builder.addFuzzyKeysData(bbpBuilder);
}
return builder.build().toByteArray();
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index 53e116e..7aa807c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -26,12 +26,12 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
/**
* A Filter that stops after the given row. There is no "RowStopFilter" because
@@ -84,7 +84,8 @@ public class InclusiveStopFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.InclusiveStopFilter.Builder builder =
FilterProtos.InclusiveStopFilter.newBuilder();
- if (this.stopRowKey != null) builder.setStopRowKey(ByteStringer.wrap(this.stopRowKey));
+ if (this.stopRowKey != null) builder.setStopRowKey(
+ UnsafeByteOperations.unsafeWrap(this.stopRowKey));
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
index 9e59e9a..328498d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
@@ -28,12 +28,11 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-
/**
* Filter to support scan multiple row key ranges. It can construct the row key ranges from the
* passed list which can be accessed by each region server.
@@ -151,10 +150,10 @@ public class MultiRowRangeFilter extends FilterBase {
if (range != null) {
FilterProtos.RowRange.Builder rangebuilder = FilterProtos.RowRange.newBuilder();
if (range.startRow != null)
- rangebuilder.setStartRow(ByteStringer.wrap(range.startRow));
+ rangebuilder.setStartRow(UnsafeByteOperations.unsafeWrap(range.startRow));
rangebuilder.setStartRowInclusive(range.startRowInclusive);
if (range.stopRow != null)
- rangebuilder.setStopRow(ByteStringer.wrap(range.stopRow));
+ rangebuilder.setStopRow(UnsafeByteOperations.unsafeWrap(range.stopRow));
rangebuilder.setStopRowInclusive(range.stopRowInclusive);
range.isScan = Bytes.equals(range.startRow, range.stopRow) ? 1 : 0;
builder.addRowRangeList(rangebuilder.build());
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
index cc97e75..9909d1f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
@@ -27,12 +27,11 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-
/**
* This filter is used for selecting only those keys with columns that matches
* a particular prefix. For example, if prefix is 'an', it will pass keys will
@@ -118,7 +117,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
FilterProtos.MultipleColumnPrefixFilter.Builder builder =
FilterProtos.MultipleColumnPrefixFilter.newBuilder();
for (byte [] element : sortedPrefixes) {
- if (element != null) builder.addSortedPrefixes(ByteStringer.wrap(element));
+ if (element != null) builder.addSortedPrefixes(UnsafeByteOperations.unsafeWrap(element));
}
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index d651f02..9bc6236 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -28,11 +28,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* Pass results that have same row prefix.
@@ -108,7 +108,7 @@ public class PrefixFilter extends FilterBase {
public byte [] toByteArray() {
FilterProtos.PrefixFilter.Builder builder =
FilterProtos.PrefixFilter.newBuilder();
- if (this.prefix != null) builder.setPrefix(ByteStringer.wrap(this.prefix));
+ if (this.prefix != null) builder.setPrefix(UnsafeByteOperations.unsafeWrap(this.prefix));
return builder.build().toByteArray();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
index 91149f5..3bb80cb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
@@ -29,12 +29,12 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
@@ -307,10 +307,10 @@ public class SingleColumnValueFilter extends FilterBase {
FilterProtos.SingleColumnValueFilter.Builder builder =
FilterProtos.SingleColumnValueFilter.newBuilder();
if (this.columnFamily != null) {
- builder.setColumnFamily(ByteStringer.wrap(this.columnFamily));
+ builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily));
}
if (this.columnQualifier != null) {
- builder.setColumnQualifier(ByteStringer.wrap(this.columnQualifier));
+ builder.setColumnQualifier(UnsafeByteOperations.unsafeWrap(this.columnQualifier));
}
HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
builder.setCompareOp(compareOp);
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
index 29ef57d..b5009e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.Encryption;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
/**
@@ -94,15 +94,15 @@ public final class EncryptionUtil {
if (cipher.getIvLength() > 0) {
iv = new byte[cipher.getIvLength()];
RNG.nextBytes(iv);
- builder.setIv(ByteStringer.wrap(iv));
+ builder.setIv(UnsafeByteOperations.unsafeWrap(iv));
}
byte[] keyBytes = key.getEncoded();
builder.setLength(keyBytes.length);
- builder.setHash(ByteStringer.wrap(Encryption.hash128(keyBytes)));
+ builder.setHash(UnsafeByteOperations.unsafeWrap(Encryption.hash128(keyBytes)));
ByteArrayOutputStream out = new ByteArrayOutputStream();
Encryption.encryptWithSubjectKey(out, new ByteArrayInputStream(keyBytes), subject,
conf, cipher, iv);
- builder.setData(ByteStringer.wrap(out.toByteArray()));
+ builder.setData(UnsafeByteOperations.unsafeWrap(out.toByteArray()));
// Build and return the protobuf message
out.reset();
builder.build().writeDelimitedTo(out);
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 5a56ee3..2f72eaa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
@@ -159,7 +160,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescript
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DynamicClassLoader;
@@ -311,7 +311,7 @@ public final class ProtobufUtil {
public static ComparatorProtos.ByteArrayComparable toByteArrayComparable(final byte [] value) {
ComparatorProtos.ByteArrayComparable.Builder builder =
ComparatorProtos.ByteArrayComparable.newBuilder();
- if (value != null) builder.setValue(ByteStringer.wrap(value));
+ if (value != null) builder.setValue(UnsafeByteOperations.unsafeWrap(value));
return builder.build();
}
@@ -949,7 +949,7 @@ public final class ProtobufUtil {
scanBuilder.setMaxVersions(scan.getMaxVersions());
for (Entry<byte[], TimeRange> cftr : scan.getColumnFamilyTimeRange().entrySet()) {
HBaseProtos.ColumnFamilyTimeRange.Builder b = HBaseProtos.ColumnFamilyTimeRange.newBuilder();
- b.setColumnFamily(ByteStringer.wrap(cftr.getKey()));
+ b.setColumnFamily(UnsafeByteOperations.unsafeWrap(cftr.getKey()));
b.setTimeRange(timeRangeToProto(cftr.getValue()));
scanBuilder.addCfTimeRange(b);
}
@@ -966,17 +966,17 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
- attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
+ attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue()));
scanBuilder.addAttribute(attributeBuilder.build());
}
}
byte[] startRow = scan.getStartRow();
if (startRow != null && startRow.length > 0) {
- scanBuilder.setStartRow(ByteStringer.wrap(startRow));
+ scanBuilder.setStartRow(UnsafeByteOperations.unsafeWrap(startRow));
}
byte[] stopRow = scan.getStopRow();
if (stopRow != null && stopRow.length > 0) {
- scanBuilder.setStopRow(ByteStringer.wrap(stopRow));
+ scanBuilder.setStopRow(UnsafeByteOperations.unsafeWrap(stopRow));
}
if (scan.hasFilter()) {
scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
@@ -985,12 +985,12 @@ public final class ProtobufUtil {
Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>>
family: scan.getFamilyMap().entrySet()) {
- columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
+ columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey()));
NavigableSet<byte []> qualifiers = family.getValue();
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte [] qualifier: qualifiers) {
- columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
+ columnBuilder.addQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
}
}
scanBuilder.addColumn(columnBuilder.build());
@@ -1112,7 +1112,7 @@ public final class ProtobufUtil {
final Get get) throws IOException {
ClientProtos.Get.Builder builder =
ClientProtos.Get.newBuilder();
- builder.setRow(ByteStringer.wrap(get.getRow()));
+ builder.setRow(UnsafeByteOperations.unsafeWrap(get.getRow()));
builder.setCacheBlocks(get.getCacheBlocks());
builder.setMaxVersions(get.getMaxVersions());
if (get.getFilter() != null) {
@@ -1120,7 +1120,7 @@ public final class ProtobufUtil {
}
for (Entry<byte[], TimeRange> cftr : get.getColumnFamilyTimeRange().entrySet()) {
HBaseProtos.ColumnFamilyTimeRange.Builder b = HBaseProtos.ColumnFamilyTimeRange.newBuilder();
- b.setColumnFamily(ByteStringer.wrap(cftr.getKey()));
+ b.setColumnFamily(UnsafeByteOperations.unsafeWrap(cftr.getKey()));
b.setTimeRange(timeRangeToProto(cftr.getValue()));
builder.addCfTimeRange(b);
}
@@ -1137,7 +1137,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
- attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
+ attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@@ -1146,11 +1146,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(ByteStringer.wrap(family.getKey()));
+ columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey()));
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte[] qualifier: qualifiers) {
- columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
+ columnBuilder.addQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
}
}
builder.addColumn(columnBuilder.build());
@@ -1190,7 +1190,7 @@ public final class ProtobufUtil {
*/
public static MutationProto toMutation(
final Increment increment, final MutationProto.Builder builder, long nonce) {
- builder.setRow(ByteStringer.wrap(increment.getRow()));
+ builder.setRow(UnsafeByteOperations.unsafeWrap(increment.getRow()));
builder.setMutateType(MutationType.INCREMENT);
builder.setDurability(toDurability(increment.getDurability()));
if (nonce != HConstants.NO_NONCE) {
@@ -1201,18 +1201,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(ByteStringer.wrap(family.getKey()));
+ columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey()));
columnBuilder.clearQualifierValue();
List<Cell> values = family.getValue();
if (values != null && values.size() > 0) {
for (Cell cell: values) {
valueBuilder.clear();
- valueBuilder.setQualifier(ByteStringer.wrap(
+ valueBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
- valueBuilder.setValue(ByteStringer.wrap(
+ valueBuilder.setValue(UnsafeByteOperations.unsafeWrap(
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
if (cell.getTagsLength() > 0) {
- valueBuilder.setTags(ByteStringer.wrap(cell.getTagsArray(),
+ valueBuilder.setTags(UnsafeByteOperations.unsafeWrap(cell.getTagsArray(),
cell.getTagsOffset(), cell.getTagsLength()));
}
columnBuilder.addQualifierValue(valueBuilder.build());
@@ -1225,7 +1225,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
- attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
+ attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@@ -1266,12 +1266,12 @@ public final class ProtobufUtil {
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
columnBuilder.clear();
- columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
+ columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey()));
for (Cell cell: family.getValue()) {
valueBuilder.clear();
- valueBuilder.setQualifier(ByteStringer.wrap(
+ valueBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
- valueBuilder.setValue(ByteStringer.wrap(
+ valueBuilder.setValue(UnsafeByteOperations.unsafeWrap(
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
valueBuilder.setTimestamp(cell.getTimestamp());
if (type == MutationType.DELETE || (type == MutationType.PUT && CellUtil.isDelete(cell))) {
@@ -1335,7 +1335,7 @@ public final class ProtobufUtil {
*/
private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
final Mutation mutation, MutationProto.Builder builder) {
- builder.setRow(ByteStringer.wrap(mutation.getRow()));
+ builder.setRow(UnsafeByteOperations.unsafeWrap(mutation.getRow()));
builder.setMutateType(type);
builder.setDurability(toDurability(mutation.getDurability()));
builder.setTimestamp(mutation.getTimeStamp());
@@ -1344,7 +1344,7 @@ public final class ProtobufUtil {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
- attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
+ attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
@@ -1491,7 +1491,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(ByteStringer.wrap(comparator.toByteArray()));
+ builder.setSerializedComparator(UnsafeByteOperations.unsafeWrap(comparator.toByteArray()));
return builder.build();
}
@@ -1555,7 +1555,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(ByteStringer.wrap(filter.toByteArray()));
+ builder.setSerializedFilter(UnsafeByteOperations.unsafeWrap(filter.toByteArray()));
return builder.build();
}
@@ -2007,15 +2007,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(ByteStringer.wrap(kv.getRowArray(), kv.getRowOffset(),
+ kvbuilder.setRow(UnsafeByteOperations.unsafeWrap(kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength()));
- kvbuilder.setFamily(ByteStringer.wrap(kv.getFamilyArray(),
+ kvbuilder.setFamily(UnsafeByteOperations.unsafeWrap(kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength()));
- kvbuilder.setQualifier(ByteStringer.wrap(kv.getQualifierArray(),
+ kvbuilder.setQualifier(UnsafeByteOperations.unsafeWrap(kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength()));
kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
kvbuilder.setTimestamp(kv.getTimestamp());
- kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(),
+ kvbuilder.setValue(UnsafeByteOperations.unsafeWrap(kv.getValueArray(), kv.getValueOffset(),
kv.getValueLength()));
return kvbuilder.build();
}
@@ -2064,10 +2064,10 @@ 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(ByteStringer.wrap(info.getTable().toBytes()))
- .setEncodedRegionName(ByteStringer.wrap(
+ .setTableName(UnsafeByteOperations.unsafeWrap(info.getTable().toBytes()))
+ .setEncodedRegionName(UnsafeByteOperations.unsafeWrap(
regionName == null ? info.getEncodedNameAsBytes() : regionName))
- .setFamilyName(ByteStringer.wrap(family))
+ .setFamilyName(UnsafeByteOperations.unsafeWrap(family))
.setStoreHomeDir(storeDir.getName()); //make relative
for (Path inputPath : inputPaths) {
builder.addCompactionInput(inputPath.getName()); //relative path
@@ -2075,7 +2075,7 @@ public final class ProtobufUtil {
for (Path outputPath : outputPaths) {
builder.addCompactionOutput(outputPath.getName());
}
- builder.setRegionName(ByteStringer.wrap(info.getRegionName()));
+ builder.setRegionName(UnsafeByteOperations.unsafeWrap(info.getRegionName()));
return builder.build();
}
@@ -2083,15 +2083,15 @@ public final class ProtobufUtil {
long flushSeqId, Map<byte[], List<Path>> committedFiles) {
FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
.setAction(action)
- .setEncodedRegionName(ByteStringer.wrap(hri.getEncodedNameAsBytes()))
- .setRegionName(ByteStringer.wrap(hri.getRegionName()))
+ .setEncodedRegionName(UnsafeByteOperations.unsafeWrap(hri.getEncodedNameAsBytes()))
+ .setRegionName(UnsafeByteOperations.unsafeWrap(hri.getRegionName()))
.setFlushSequenceNumber(flushSeqId)
- .setTableName(ByteStringer.wrap(hri.getTable().getName()));
+ .setTableName(UnsafeByteOperations.unsafeWrap(hri.getTable().getName()));
for (Map.Entry<byte[], List<Path>> entry : committedFiles.entrySet()) {
WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder builder =
WALProtos.FlushDescriptor.StoreFlushDescriptor.newBuilder()
- .setFamilyName(ByteStringer.wrap(entry.getKey()))
+ .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey()))
.setStoreHomeDir(Bytes.toString(entry.getKey())); //relative to region
if (entry.getValue() != null) {
for (Path path : entry.getValue()) {
@@ -2129,15 +2129,15 @@ public final class ProtobufUtil {
Map<byte[], List<Path>> storeFiles) {
RegionEventDescriptor.Builder desc = RegionEventDescriptor.newBuilder()
.setEventType(eventType)
- .setTableName(ByteStringer.wrap(tableNameAsBytes))
- .setEncodedRegionName(ByteStringer.wrap(encodedNameAsBytes))
- .setRegionName(ByteStringer.wrap(regionNameAsBytes))
+ .setTableName(UnsafeByteOperations.unsafeWrap(tableNameAsBytes))
+ .setEncodedRegionName(UnsafeByteOperations.unsafeWrap(encodedNameAsBytes))
+ .setRegionName(UnsafeByteOperations.unsafeWrap(regionNameAsBytes))
.setLogSequenceNumber(seqId)
.setServer(toServerName(server));
for (Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
- .setFamilyName(ByteStringer.wrap(entry.getKey()))
+ .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey()))
.setStoreHomeDir(Bytes.toString(entry.getKey()));
for (Path path : entry.getValue()) {
builder.addStoreFile(path.getName());
@@ -2214,8 +2214,8 @@ public final class ProtobufUtil {
public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
return HBaseProtos.TableName.newBuilder()
- .setNamespace(ByteStringer.wrap(tableName.getNamespace()))
- .setQualifier(ByteStringer.wrap(tableName.getQualifier())).build();
+ .setNamespace(UnsafeByteOperations.unsafeWrap(tableName.getNamespace()))
+ .setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier())).build();
}
public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {
@@ -2484,7 +2484,7 @@ public final class ProtobufUtil {
for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
WALProtos.StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
- .setFamilyName(ByteStringer.wrap(entry.getKey()))
+ .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey()))
.setStoreHomeDir(Bytes.toString(entry.getKey())); // relative to region
for (Path path : entry.getValue()) {
String name = path.getName();
@@ -2697,11 +2697,11 @@ public final class ProtobufUtil {
*/
public static ColumnFamilySchema convertToColumnFamilySchema(HColumnDescriptor hcd) {
ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
- builder.setName(ByteStringer.wrap(hcd.getName()));
+ builder.setName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
for (Map.Entry<Bytes, Bytes> e : hcd.getValues().entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
- aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
- aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
+ aBuilder.setFirst(UnsafeByteOperations.unsafeWrap(e.getKey().get()));
+ aBuilder.setSecond(UnsafeByteOperations.unsafeWrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (Map.Entry<String, String> e : hcd.getConfiguration().entrySet()) {
@@ -2742,8 +2742,8 @@ public final class ProtobufUtil {
builder.setTableName(toProtoTableName(htd.getTableName()));
for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
- aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
- aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
+ aBuilder.setFirst(UnsafeByteOperations.unsafeWrap(e.getKey().get()));
+ aBuilder.setSecond(UnsafeByteOperations.unsafeWrap(e.getValue().get()));
builder.addAttributes(aBuilder.build());
}
for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
@@ -2948,7 +2948,7 @@ public final class ProtobufUtil {
ClusterStatusProtos.RegionState rs = rit.convert();
RegionSpecifier.Builder spec =
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
- spec.setValue(ByteStringer.wrap(rit.getRegion().getRegionName()));
+ spec.setValue(UnsafeByteOperations.unsafeWrap(rit.getRegion().getRegionName()));
RegionInTransition pbRIT =
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
@@ -3025,7 +3025,7 @@ public final class ProtobufUtil {
RegionSpecifier region = RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
- builder.addFamily(ByteStringer.wrap(family));
+ builder.addFamily(UnsafeByteOperations.unsafeWrap(family));
return builder.build();
}
@@ -3088,7 +3088,7 @@ public final class ProtobufUtil {
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
if (splitPoint != null) {
- builder.setSplitPoint(ByteStringer.wrap(splitPoint));
+ builder.setSplitPoint(UnsafeByteOperations.unsafeWrap(splitPoint));
}
return builder.build();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index baec3e4..3f91ee0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -105,7 +106,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOr
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
@@ -160,14 +160,14 @@ public final class RequestConverter {
builder.setRegion(region);
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
- mutateBuilder.setRow(ByteStringer.wrap(row));
+ mutateBuilder.setRow(UnsafeByteOperations.unsafeWrap(row));
mutateBuilder.setMutateType(MutationType.INCREMENT);
mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
- columnBuilder.setFamily(ByteStringer.wrap(family));
+ columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family));
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
- valueBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(amount)));
- valueBuilder.setQualifier(ByteStringer.wrap(qualifier));
+ valueBuilder.setValue(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(amount)));
+ valueBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build());
if (nonce != HConstants.NO_NONCE) {
@@ -549,8 +549,8 @@ public final class RequestConverter {
if (userToken != null) {
protoDT =
ClientProtos.DelegationToken.newBuilder()
- .setIdentifier(ByteStringer.wrap(userToken.getIdentifier()))
- .setPassword(ByteStringer.wrap(userToken.getPassword()))
+ .setIdentifier(UnsafeByteOperations.unsafeWrap(userToken.getIdentifier()))
+ .setPassword(UnsafeByteOperations.unsafeWrap(userToken.getPassword()))
.setKind(userToken.getKind().toString())
.setService(userToken.getService().toString()).build();
}
@@ -559,7 +559,7 @@ public final class RequestConverter {
new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>(familyPaths.size());
for(Pair<byte[], String> el: familyPaths) {
protoFamilyPaths.add(ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder()
- .setFamily(ByteStringer.wrap(el.getFirst()))
+ .setFamily(UnsafeByteOperations.unsafeWrap(el.getFirst()))
.setPath(el.getSecond()).build());
}
@@ -619,7 +619,7 @@ public final class RequestConverter {
exec.getRequest().toByteArray());
regionActionBuilder.addAction(actionBuilder.setServiceCall(
ClientProtos.CoprocessorServiceCall.newBuilder()
- .setRow(ByteStringer.wrap(exec.getRow()))
+ .setRow(UnsafeByteOperations.unsafeWrap(exec.getRow()))
.setServiceName(exec.getMethod().getService().getFullName())
.setMethodName(exec.getMethod().getName())
.setRequest(value)));
@@ -702,7 +702,7 @@ public final class RequestConverter {
exec.getRequest().toByteArray());
builder.addAction(actionBuilder.setServiceCall(
ClientProtos.CoprocessorServiceCall.newBuilder()
- .setRow(ByteStringer.wrap(exec.getRow()))
+ .setRow(UnsafeByteOperations.unsafeWrap(exec.getRow()))
.setServiceName(exec.getMethod().getService().getFullName())
.setMethodName(exec.getMethod().getName())
.setRequest(value)));
@@ -875,7 +875,7 @@ public final class RequestConverter {
builder.setRegion(region);
builder.setMajor(major);
if (family != null) {
- builder.setFamily(ByteStringer.wrap(family));
+ builder.setFamily(UnsafeByteOperations.unsafeWrap(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(ByteStringer.wrap(value));
+ regionBuilder.setValue(UnsafeByteOperations.unsafeWrap(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(ByteStringer.wrap(row));
- builder.setFamily(ByteStringer.wrap(family));
- builder.setQualifier(ByteStringer.wrap(qualifier));
+ builder.setRow(UnsafeByteOperations.unsafeWrap(row));
+ builder.setFamily(UnsafeByteOperations.unsafeWrap(family));
+ builder.setQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
builder.setComparator(ProtobufUtil.toComparator(comparator));
builder.setCompareType(compareType);
return builder.build();
@@ -997,7 +997,7 @@ public final class RequestConverter {
final long nonce) {
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
- builder.setColumnName(ByteStringer.wrap(columnName));
+ builder.setColumnName(UnsafeByteOperations.unsafeWrap(columnName));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
@@ -1187,7 +1187,7 @@ public final class RequestConverter {
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
if (splitKeys != null) {
for (byte [] splitKey : splitKeys) {
- builder.addSplitKeys(ByteStringer.wrap(splitKey));
+ builder.addSplitKeys(UnsafeByteOperations.unsafeWrap(splitKey));
}
}
builder.setNonceGroup(nonceGroup);
@@ -1401,7 +1401,7 @@ public final class RequestConverter {
public static GetLastFlushedSequenceIdRequest buildGetLastFlushedSequenceIdRequest(
byte[] regionName) {
return GetLastFlushedSequenceIdRequest.newBuilder().setRegionName(
- ByteStringer.wrap(regionName)).build();
+ UnsafeByteOperations.unsafeWrap(regionName)).build();
}
/**
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 8018bd7..cf11641 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -42,16 +42,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.AuthUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
-import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@@ -76,7 +76,6 @@ import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.zookeeper.server.ZooKeeperSaslServer;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
/**
* Internal HBase utility class for ZooKeeper.
@@ -2107,7 +2106,7 @@ public class ZKUtil {
for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
byte[] columnFamilyName = e.getKey();
Long curSeqId = e.getValue();
- storeSequenceIdBuilder.setFamilyName(ByteStringer.wrap(columnFamilyName));
+ storeSequenceIdBuilder.setFamilyName(UnsafeByteOperations.unsafeWrap(columnFamilyName));
storeSequenceIdBuilder.setSequenceId(curSeqId);
regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
storeSequenceIdBuilder.clear();
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
index df6de42..41c9a56 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
@@ -77,7 +77,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
@@ -93,6 +92,7 @@ import com.google.common.base.Stopwatch;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/**
* Test client behavior w/o setting up a cluster.
@@ -509,7 +509,7 @@ public class TestClientNoCluster extends Configured implements Tool {
if (max <= 0) break;
if (++count > max) break;
HRegionInfo hri = e.getValue().getFirst();
- ByteString row = ByteStringer.wrap(hri.getRegionName());
+ ByteString row = UnsafeByteOperations.unsafeWrap(hri.getRegionName());
resultBuilder.clear();
resultBuilder.addCell(getRegionInfo(row, hri));
resultBuilder.addCell(getServer(row, e.getValue().getSecond()));
@@ -556,11 +556,11 @@ public class TestClientNoCluster extends Configured implements Tool {
}
private final static ByteString CATALOG_FAMILY_BYTESTRING =
- ByteStringer.wrap(HConstants.CATALOG_FAMILY);
+ UnsafeByteOperations.unsafeWrap(HConstants.CATALOG_FAMILY);
private final static ByteString REGIONINFO_QUALIFIER_BYTESTRING =
- ByteStringer.wrap(HConstants.REGIONINFO_QUALIFIER);
+ UnsafeByteOperations.unsafeWrap(HConstants.REGIONINFO_QUALIFIER);
private final static ByteString SERVER_QUALIFIER_BYTESTRING =
- ByteStringer.wrap(HConstants.SERVER_QUALIFIER);
+ UnsafeByteOperations.unsafeWrap(HConstants.SERVER_QUALIFIER);
static CellProtos.Cell.Builder getBaseCellBuilder(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = CellProtos.Cell.newBuilder();
@@ -573,7 +573,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(ByteStringer.wrap(hri.toByteArray()));
+ cellBuilder.setValue(UnsafeByteOperations.unsafeWrap(hri.toByteArray()));
return cellBuilder.build();
}
@@ -586,9 +586,10 @@ public class TestClientNoCluster extends Configured implements Tool {
static CellProtos.Cell getStartCode(final ByteString row) {
CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row);
- cellBuilder.setQualifier(ByteStringer.wrap(HConstants.STARTCODE_QUALIFIER));
+ cellBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(HConstants.STARTCODE_QUALIFIER));
// TODO:
- cellBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(META_SERVERNAME.getStartcode())));
+ cellBuilder.setValue(UnsafeByteOperations.unsafeWrap(
+ Bytes.toBytes(META_SERVERNAME.getStartcode())));
return cellBuilder.build();
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
index c2985fd..05e8e09 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -17,13 +17,13 @@
*/
package org.apache.hadoop.hbase.procedure2;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ProcedureState;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.shaded.util.ByteStringer;
import org.apache.hadoop.hbase.util.NonceKey;
/**
@@ -60,7 +60,7 @@ public final class ProcedureUtil {
}
if (procInfo.hasResultData()) {
- builder.setResult(ByteStringer.wrap(procInfo.getResult()));
+ builder.setResult(UnsafeByteOperations.unsafeWrap(procInfo.getResult()));
}
return builder.build();
http://git-wip-us.apache.org/repos/asf/hbase/blob/32be831c/hbase-protocol-shaded/README.txt
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/README.txt b/hbase-protocol-shaded/README.txt
index 387ada9..5a4b83b 100644
--- a/hbase-protocol-shaded/README.txt
+++ b/hbase-protocol-shaded/README.txt
@@ -1,41 +1,44 @@
-These are the protobuf definition files used by core hbase. This modules
-does shading of all to do with protobuf. All of core relies on this module.
-All core references in core to protobuf are to the protobuf this module
-includes but offset by the package prefix of org.apache.hadoop.hbase.shaded.*
-as in org.apache.hadoop.hbase.shaded.protobuf.generated.* and
-org.apache.hadoop.hbase.shaded.com.google.protobuf.*.
-
-NOTE: the .protos in here are copied in an adjacent module, hbase-protocol.
-There they are non-shaded. If you make changes here, consider making them
-over in the adjacent module too. Be careful, the .proto files are not
-exactly the same; they differ in one line at least -- the location the file
-gets generated to; i.e. those in here get the extra 'shaded' in their
-package name.
-
-The produced java classes are generated and then checked in. The reasoning
-is that they change infrequently.
-
-To regenerate the classes after making definition file changes, ensure first that
-the protobuf protoc tool is in your $PATH. You may need to download it and build
-it first; its part of the protobuf package. For example, if using v2.5.0 of
-protobuf, it is obtainable from here:
-
- https://github.com/google/protobuf/releases/tag/v2.5.0
-
-HBase uses hadoop-maven-plugins:protoc goal to invoke the protoc command. You can
-compile the protoc definitions by invoking maven with profile compile-protobuf or
-passing in compile-protobuf property.
-
-mvn compile -Dcompile-protobuf
-or
-mvn compile -Pcompile-protobuf
+Please read carefully as the 'menu options' have changed.
+
+This module has proto files used by core. These protos
+overlap with protos that are used by coprocessor endpoints
+(CPEP) in the module hbase-protocol. So the core versions have
+a different name, the generated classes are relocated
+-- i.e. shaded -- to a new location; they are moved from
+org.apache.hadoop.hbase.* to org.apache.hadoop.hbase.shaded.
+
+This module also includes the protobuf that hbase core depends
+on again relocated to live at an offset of
+org.apache.hadoop.hbase.shaded so as to avoid clashes with other
+versions of protobuf resident on our CLASSPATH included,
+transitively or otherwise, by dependencies: i.e. the shaded
+protobuf Message class is at
+org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
+rather than at com.google.protobuf.Message.
+
+Below we describe how to generate the java files for this
+module. Run this step any time you change the proto files
+in this module or if you change the protobuf version. If you
+add a new file, be sure to add mention of the proto in the
+pom.xml (scroll till you see the listing of protos to consider).
+
+First ensure that the appropriate protobuf protoc tool is in
+your $PATH (or pass -Dprotoc.path=PATH_TO_PROTOC when running
+the below mvn commands). You may need to download protobuf and
+build protoc first.
+
+Run:
+
+ $ mvn install -Dgenerate-shaded-classes
-You may also want to define protoc.path for the protoc binary
+or
-mvn compile -Dcompile-protobuf -Dprotoc.path=/opt/local/bin/protoc
+ $ mvn install -Pgenerate-shaded-classes
-If you have added a new proto file, you should add it to the pom.xml file first.
-Other modules also support the maven profile.
+to build and trigger the special generate-shaded-classes
+profile. When finished, the content of
+src/main/java/org/apache/hadoop/hbase/shaded will have
+been updated. Check in the changes.
-After you've done the above, check it in and then check it in (or post a patch
-on a JIRA with your definition file changes and the generated files).
+See the pom.xml under the generate-shaded-classes profile
+for more info on how this step works.