You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/10/07 19:15:16 UTC

[48/77] [abbrv] [partial] hbase git commit: HBASE-15638 Shade protobuf Which includes

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
index 3cbb7b9..b59398b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
@@ -24,11 +24,12 @@ import java.nio.ByteBuffer;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A binary comparator which lexicographically compares against the specified
@@ -36,8 +37,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class BinaryComparator extends ByteArrayComparable {
-
+public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayComparable {
   /**
    * Constructor
    * @param value value
@@ -62,7 +62,7 @@ public class BinaryComparator extends ByteArrayComparable {
   public byte [] toByteArray() {
     ComparatorProtos.BinaryComparator.Builder builder =
       ComparatorProtos.BinaryComparator.newBuilder();
-    builder.setComparable(super.convert());
+    builder.setComparable(ProtobufUtil.toByteArrayComparable(this.value));
     return builder.build().toByteArray();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
index a26edbc..01cb769 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
@@ -24,11 +24,12 @@ import java.nio.ByteBuffer;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A comparator which compares against a specified byte array, but only compares
@@ -67,7 +68,7 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
   public byte [] toByteArray() {
     ComparatorProtos.BinaryPrefixComparator.Builder builder =
       ComparatorProtos.BinaryPrefixComparator.newBuilder();
-    builder.setComparable(super.convert());
+    builder.setComparable(ProtobufUtil.toByteArrayComparable(this.value));
     return builder.build().toByteArray();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
index db51df7..dac8864 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
@@ -24,9 +24,10 @@ import java.nio.ByteBuffer;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A bit comparator which performs the specified bitwise operation on each of the bytes
@@ -72,7 +73,7 @@ public class BitComparator extends ByteArrayComparable {
   public byte [] toByteArray() {
     ComparatorProtos.BitComparator.Builder builder =
       ComparatorProtos.BitComparator.newBuilder();
-    builder.setComparable(super.convert());
+    builder.setComparable(ProtobufUtil.toByteArrayComparable(this.value));
     ComparatorProtos.BitComparator.BitwiseOp bitwiseOpPb =
       ComparatorProtos.BitComparator.BitwiseOp.valueOf(bitOperator.name());
     builder.setBitwiseOp(bitwiseOpPb);

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
index c747b00..3ae20a1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Simple filter that returns first N columns on row only.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 e5ec412..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
@@ -27,12 +27,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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+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/95c1dc93/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 ff6e8e2..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
@@ -28,13 +28,13 @@ 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+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/95c1dc93/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 04682c5..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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import 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/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index 0029d44..6b64e0d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
+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.util.Bytes;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 f7c6f26..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
@@ -30,13 +30,14 @@ 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+
+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
@@ -221,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/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
index b3f9a1a..8dfd2ca 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
@@ -26,10 +26,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * <p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index 1ea3c80..901e567 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -29,10 +29,9 @@ 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Implementation of {@link Filter} that represents an ordered List of Filters
@@ -406,8 +405,7 @@ final public class FilterList extends Filter {
 
     List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
     try {
-      List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.Filter> filtersList =
-          proto.getFiltersList();
+      List<FilterProtos.Filter> filtersList = proto.getFiltersList();
       int listSize = filtersList.size();
       for (int i = 0; i < listSize; i++) {
         rowFilters.add(ProtobufUtil.toFilter(filtersList.get(i)));

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
index 617cd7a..d25a642 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
@@ -25,10 +25,9 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This is a Filter wrapper class which is used in the server side. Some filter

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
index 80a1deb..3549304 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that will only return the first KV from each row.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 2e9510f..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
@@ -26,12 +26,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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+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/95c1dc93/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 f52137d..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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.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 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/95c1dc93/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 5dcb50d..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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import 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/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 1a0d2af..5ed5802 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -29,11 +29,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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that will only return the key component of each KV (the value will

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
index 6bd4b0f..8bcc7b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java
@@ -20,14 +20,16 @@ package org.apache.hadoop.hbase.filter;
 
 import java.nio.ByteBuffer;
 
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A long comparator which numerical compares against the specified byte array
@@ -61,7 +63,7 @@ public class LongComparator extends ByteArrayComparable {
     public byte [] toByteArray() {
         ComparatorProtos.LongComparator.Builder builder =
                 ComparatorProtos.LongComparator.newBuilder();
-        builder.setComparable(super.convert());
+        builder.setComparable(ProtobufUtil.toByteArrayComparable(this.value));
         return builder.build().toByteArray();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 33f035c..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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.util.Bytes;
 
-import 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/95c1dc93/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 0c14649..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.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.util.Bytes;
 
-import 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/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
index 160232f..0d60e2e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
@@ -24,9 +24,9 @@ import java.nio.ByteBuffer;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A binary comparator which lexicographically compares against the specified

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
index f12fac8..2b91b7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 /**
  * Implementation of Filter interface that limits results to a specific page
  * size. It terminates scanning once the number of filter-passed rows is &gt;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 e7b91e1..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
@@ -26,13 +26,13 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+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/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index 3aa3558..cc240f8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -26,10 +26,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter based on the column qualifier. It takes an

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
index decdc78..177ed4d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that includes rows based on a chance.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
index 70dd1f9..3f05901 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
@@ -28,7 +28,7 @@ import org.apache.commons.logging.LogFactory;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.jcodings.Encoding;
 import org.jcodings.EncodingDB;
@@ -38,7 +38,7 @@ import org.joni.Option;
 import org.joni.Regex;
 import org.joni.Syntax;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This comparator is for use with {@link CompareFilter} implementations, such

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
index 559eff8..4f91f8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
@@ -26,10 +26,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter based on the key. It takes an operator

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
index d030fd2..192fd97 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
@@ -30,10 +30,9 @@ 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A {@link Filter} that checks a single column value, but does not emit the

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 df4e482..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,15 +29,15 @@ 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.util.ByteStringer;
+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.util.Bytes;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
@@ -74,7 +74,7 @@ public class SingleColumnValueFilter extends FilterBase {
   protected byte [] columnFamily;
   protected byte [] columnQualifier;
   protected CompareOp compareOp;
-  protected ByteArrayComparable comparator;
+  protected org.apache.hadoop.hbase.filter.ByteArrayComparable comparator;
   protected boolean foundColumn = false;
   protected boolean matchedColumn = false;
   protected boolean filterIfMissing = false;
@@ -96,7 +96,7 @@ public class SingleColumnValueFilter extends FilterBase {
    */
   public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
       final CompareOp compareOp, final byte[] value) {
-    this(family, qualifier, compareOp, new BinaryComparator(value));
+    this(family, qualifier, compareOp, new org.apache.hadoop.hbase.filter.BinaryComparator(value));
   }
 
   /**
@@ -114,7 +114,8 @@ public class SingleColumnValueFilter extends FilterBase {
    * @param comparator Comparator to use.
    */
   public SingleColumnValueFilter(final byte [] family, final byte [] qualifier,
-      final CompareOp compareOp, final ByteArrayComparable comparator) {
+      final CompareOp compareOp,
+      final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) {
     this.columnFamily = family;
     this.columnQualifier = qualifier;
     this.compareOp = compareOp;
@@ -131,7 +132,8 @@ public class SingleColumnValueFilter extends FilterBase {
    * @param latestVersionOnly
    */
   protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier,
-      final CompareOp compareOp, ByteArrayComparable comparator, final boolean filterIfMissing,
+      final CompareOp compareOp, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator,
+      final boolean filterIfMissing,
       final boolean latestVersionOnly) {
     this(family, qualifier, compareOp, comparator);
     this.filterIfMissing = filterIfMissing;
@@ -148,7 +150,7 @@ public class SingleColumnValueFilter extends FilterBase {
   /**
    * @return the comparator
    */
-  public ByteArrayComparable getComparator() {
+  public org.apache.hadoop.hbase.filter.ByteArrayComparable getComparator() {
     return comparator;
   }
 
@@ -277,7 +279,7 @@ public class SingleColumnValueFilter extends FilterBase {
     byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
     byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
     CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(2));
-    ByteArrayComparable comparator = ParseFilter.createComparator(
+    org.apache.hadoop.hbase.filter.ByteArrayComparable comparator = ParseFilter.createComparator(
       ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
 
     if (comparator instanceof RegexStringComparator ||
@@ -305,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);
@@ -343,7 +345,7 @@ public class SingleColumnValueFilter extends FilterBase {
 
     final CompareOp compareOp =
       CompareOp.valueOf(proto.getCompareOp().name());
-    final ByteArrayComparable comparator;
+    final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator;
     try {
       comparator = ProtobufUtil.toComparator(proto.getComparator());
     } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
index 3aced13..5461011 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
@@ -25,10 +25,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A wrapper filter that filters an entire row if any of the Cell checks do

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
index d36b158..d30d057 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
@@ -22,10 +22,10 @@ import java.util.Locale;
 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.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
index f0e5afe..2f10d9a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
@@ -27,10 +27,10 @@ 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.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Filter that returns only cells whose timestamp (version) is

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
index 2f679f0..c0dacaf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
@@ -26,10 +26,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter based on column value. It takes an

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
index 93b4a00..8738962 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
@@ -25,10 +25,9 @@ import org.apache.hadoop.hbase.Cell;
 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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 990ffe0..4df6786 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -26,14 +26,13 @@ import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel;
+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.RpcCallback;
 
 import io.netty.util.HashedWheelTimer;
 
@@ -402,7 +401,6 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr);
     Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType,
         hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback<Call>() {
-
           @Override
           public void run(Call call) {
             counter.decrementAndGet();
@@ -451,6 +449,26 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
       }
     }
   }
+  /**
+   * Configure an hbase rpccontroller
+   * @param controller to configure
+   * @param channelOperationTimeout timeout for operation
+   * @return configured controller
+   */
+  static HBaseRpcController configureHBaseRpcController(
+      RpcController controller, int channelOperationTimeout) {
+    HBaseRpcController hrc;
+    if (controller != null && controller instanceof HBaseRpcController) {
+      hrc = (HBaseRpcController) controller;
+      if (!hrc.hasCallTimeout()) {
+        hrc.setCallTimeout(channelOperationTimeout);
+      }
+    } else {
+      hrc = new HBaseRpcControllerImpl();
+      hrc.setCallTimeout(channelOperationTimeout);
+    }
+    return hrc;
+  }
 
   protected abstract void closeInternal();
 
@@ -553,7 +571,8 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
   /**
    * Async rpc channel that goes via hbase rpc.
    */
-  public static class RpcChannelImplementation extends AbstractRpcChannel implements RpcChannel {
+  public static class RpcChannelImplementation extends AbstractRpcChannel implements
+      RpcChannel {
 
     protected RpcChannelImplementation(AbstractRpcClient<?> rpcClient, InetSocketAddress addr,
         User ticket, int rpcTimeout) throws UnknownHostException {
@@ -561,8 +580,8 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     }
 
     @Override
-    public void callMethod(MethodDescriptor md, RpcController controller, Message param,
-        Message returnType, RpcCallback<Message> done) {
+    public void callMethod(Descriptors.MethodDescriptor md, RpcController controller,
+        Message param, Message returnType, RpcCallback<Message> done) {
       // This method does not throw any exceptions, so the caller must provide a
       // HBaseRpcController which is used to pass the exceptions.
       this.rpcClient.callMethod(md,
@@ -571,4 +590,4 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
         param, returnType, ticket, addr, done);
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
index 523ca55..fbd92856 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -68,4 +68,4 @@ public class BlockingRpcCallback<R> implements RpcCallback<R> {
     }
     return result;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index 528b726..5ae5508 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -24,9 +24,9 @@ import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
 
-import com.google.protobuf.Message;
-import com.google.protobuf.Message.Builder;
-import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message.Builder;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -59,12 +59,12 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
 import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index a6203d5..04d7238 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-
 import io.netty.util.Timeout;
 
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
index 45c3700..7ed1f7d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java
@@ -30,5 +30,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public interface CoprocessorRpcChannel extends RpcChannel, BlockingRpcChannel {
-}
\ No newline at end of file
+public interface CoprocessorRpcChannel extends RpcChannel, BlockingRpcChannel {}
+// This Interface is part of our public, client-facing API!!!
+// This belongs in client package but it is exposed in our public API so we cannot relocate.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java
index 63ff3e8..9dfa334 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java
@@ -19,17 +19,39 @@
 
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Descriptors.ServiceDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 /**
- * Utilities for handling coprocessor service calls.
+ * Utilities for handling coprocessor rpc service calls.
  */
 @InterfaceAudience.Private
 public final class CoprocessorRpcUtils {
+  private static final Log LOG = LogFactory.getLog(CoprocessorRpcUtils.class);
   /**
    * We assume that all HBase protobuf services share a common package name
    * (defined in the .proto files).
@@ -58,15 +80,166 @@ public final class CoprocessorRpcUtils {
     return service.getFullName();
   }
 
+  public static CoprocessorServiceRequest getCoprocessorServiceRequest(
+      final Descriptors.MethodDescriptor method, final Message request) {
+    return getCoprocessorServiceRequest(method, request, HConstants.EMPTY_BYTE_ARRAY,
+        HConstants.EMPTY_BYTE_ARRAY);
+  }
+
+  public static CoprocessorServiceRequest getCoprocessorServiceRequest(
+      final Descriptors.MethodDescriptor method, final Message request, final byte [] row,
+      final byte [] regionName) {
+    return CoprocessorServiceRequest.newBuilder().setCall(
+        getCoprocessorServiceCall(method, request, row)).
+          setRegion(RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
+  }
+
+  private static CoprocessorServiceCall getCoprocessorServiceCall(
+      final Descriptors.MethodDescriptor method, final Message request, final byte [] row) {
+    return CoprocessorServiceCall.newBuilder()
+    .setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFrom(row))
+    .setServiceName(CoprocessorRpcUtils.getServiceName(method.getService()))
+    .setMethodName(method.getName())
+    // TODO!!!!! Come back here after!!!!! This is a double copy of the request if I read
+    // it right copying from non-shaded to shaded version!!!!!! FIXXXXX!!!!!
+    .setRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.
+          copyFrom(request.toByteArray())).build();
+  }
+
+  public static MethodDescriptor getMethodDescriptor(final String methodName,
+      final ServiceDescriptor serviceDesc)
+  throws UnknownProtocolException {
+    Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
+    if (methodDesc == null) {
+      throw new UnknownProtocolException("Unknown method " + methodName + " called on service " +
+          serviceDesc.getFullName());
+    }
+    return methodDesc;
+  }
+
+  public static Message getRequest(Service service,
+      Descriptors.MethodDescriptor methodDesc,
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString shadedRequest)
+  throws IOException {
+    Message.Builder builderForType =
+        service.getRequestPrototype(methodDesc).newBuilderForType();
+    org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builderForType,
+        // TODO: COPY FROM SHADED TO NON_SHADED. DO I HAVE TOO?
+        shadedRequest.toByteArray());
+    return builderForType.build();
+  }
+
+  public static Message getResponse(
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse
+        result,
+      com.google.protobuf.Message responsePrototype)
+  throws IOException {
+    Message response;
+    if (result.getValue().hasValue()) {
+      Message.Builder builder = responsePrototype.newBuilderForType();
+      builder.mergeFrom(result.getValue().getValue().newInput());
+      response = builder.build();
+    } else {
+      response = responsePrototype.getDefaultInstanceForType();
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Master Result is value=" + response);
+    }
+    return response;
+  }
+
+  public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.
+      CoprocessorServiceResponse getResponse(final Message result, final byte [] regionName) {
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.
+      CoprocessorServiceResponse.Builder builder =
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.
+        newBuilder();
+    builder.setRegion(RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME,
+      regionName));
+    // TODO: UGLY COPY IN HERE!!!!
+    builder.setValue(builder.getValueBuilder().setName(result.getClass().getName())
+        .setValue(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.
+            copyFrom(result.toByteArray())));
+    return builder.build();
+  }
+
+  /**
+   * Simple {@link RpcCallback} implementation providing a
+   * {@link java.util.concurrent.Future}-like {@link BlockingRpcCallback#get()} method, which
+   * will block util the instance's {@link BlockingRpcCallback#run(Object)} method has been called.
+   * {@code R} is the RPC response type that will be passed to the {@link #run(Object)} method.
+   */
+  @InterfaceAudience.Private
+  // Copy of BlockingRpcCallback but deriving from RpcCallback non-shaded.
+  public static class BlockingRpcCallback<R> implements RpcCallback<R> {
+    private R result;
+    private boolean resultSet = false;
+
+    /**
+     * Called on completion of the RPC call with the response object, or {@code null} in the case of
+     * an error.
+     * @param parameter the response object or {@code null} if an error occurred
+     */
+    @Override
+    public void run(R parameter) {
+      synchronized (this) {
+        result = parameter;
+        resultSet = true;
+        this.notifyAll();
+      }
+    }
+
+    /**
+     * Returns the parameter passed to {@link #run(Object)} or {@code null} if a null value was
+     * passed.  When used asynchronously, this method will block until the {@link #run(Object)}
+     * method has been called.
+     * @return the response object or {@code null} if no response was passed
+     */
+    public synchronized R get() throws IOException {
+      while (!resultSet) {
+        try {
+          this.wait();
+        } catch (InterruptedException ie) {
+          InterruptedIOException exception = new InterruptedIOException(ie.getMessage());
+          exception.initCause(ie);
+          throw exception;
+        }
+      }
+      return result;
+    }
+  }
+
+  /**
+   * Stores an exception encountered during RPC invocation so it can be passed back
+   * through to the client.
+   * @param controller the controller instance provided by the client when calling the service
+   * @param ioe the exception encountered
+   */
+  public static void setControllerException(RpcController controller, IOException ioe) {
+    if (controller == null) {
+      return;
+    }
+    if (controller instanceof org.apache.hadoop.hbase.ipc.ServerRpcController) {
+      ((ServerRpcController)controller).setFailedOn(ioe);
+    } else {
+      controller.setFailed(StringUtils.stringifyException(ioe));
+    }
+  }
+
   /**
-   * Returns a service call instance for the given coprocessor request.
+   * Retreivies exception stored during RPC invocation.
+   * @param controller the controller instance provided by the client when calling the service
+   * @return exception if any, or null; Will return DoNotRetryIOException for string represented
+   * failure causes in controller.
    */
-  public static ClientProtos.CoprocessorServiceCall buildServiceCall(byte[] row,
-      Descriptors.MethodDescriptor method, Message request) {
-    return ClientProtos.CoprocessorServiceCall.newBuilder()
-        .setRow(ByteStringer.wrap(row))
-        .setServiceName(CoprocessorRpcUtils.getServiceName(method.getService()))
-        .setMethodName(method.getName())
-        .setRequest(request.toByteString()).build();
+  @Nullable
+  public static IOException getControllerException(RpcController controller) throws IOException {
+    if (controller == null || !controller.failed()) {
+      return null;
+    }
+    if (controller instanceof ServerRpcController) {
+      return ((ServerRpcController)controller).getFailedOn();
+    }
+    return new DoNotRetryIOException(controller.errorText());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java
index 9f9c636..f899409 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
 import java.io.IOException;
 
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * Simple delegating controller for use with the {@link RpcControllerFactory} to help override
- * standard behavior of a {@link HBaseRpcController}.
+ * standard behavior of a {@link HBaseRpcController}. Used testing.
  */
 @InterfaceAudience.Private
 public class DelegatingHBaseRpcController implements HBaseRpcController {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
index 2c4b335..71ce70a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
index a976473..8ceac64 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
 import java.io.IOException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index f66350b..2ee141c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -30,10 +30,10 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.ipc.RemoteException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
deleted file mode 100644
index a6384e3..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcController;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-
-/**
- * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
- * against the active master.  An instance of this class may be obtained
- * by calling {@link org.apache.hadoop.hbase.client.HBaseAdmin#coprocessorService()},
- * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to call the endpoint
- * methods.
- * @see org.apache.hadoop.hbase.client.HBaseAdmin#coprocessorService()
- */
-@InterfaceAudience.Private
-public class MasterCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
-  private static final Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class);
-
-  private final ClusterConnection connection;
-
-  public MasterCoprocessorRpcChannel(ClusterConnection conn) {
-    this.connection = conn;
-  }
-
-  @Override
-  protected Message callExecService(RpcController controller, Descriptors.MethodDescriptor method,
-                                  Message request, Message responsePrototype)
-      throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: "+method.getName()+", "+request.toString());
-    }
-
-    final ClientProtos.CoprocessorServiceCall call =
-        CoprocessorRpcUtils.buildServiceCall(HConstants.EMPTY_BYTE_ARRAY, method, request);
-
-    // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
-    CoprocessorServiceResponse result = ProtobufUtil.execService(controller,
-      connection.getMaster(), call);
-    Message response = null;
-    if (result.getValue().hasValue()) {
-      Message.Builder builder = responsePrototype.newBuilderForType();
-      ProtobufUtil.mergeFrom(builder, result.getValue().getValue());
-      response = builder.build();
-    } else {
-      response = responsePrototype.getDefaultInstanceForType();
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Master Result is value=" + response);
-    }
-    return response;
-  }
-}
\ No newline at end of file